From 0092abb47a0f9fdc716d5dfc1c591ddb45de8c98 Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Tue, 5 May 2015 18:32:16 -0700 Subject: [PATCH 01/75] Some minor cleanup after SPARK-4550. JoshRosen this PR addresses the comments you left on #4450 after it got merged. Author: Sandy Ryza Closes #5916 from sryza/sandy-spark-4550-cleanup and squashes the following commits: dee3d85 [Sandy Ryza] Some minor cleanup after SPARK-4550. --- .../org/apache/spark/storage/BlockObjectWriter.scala | 9 ++------- .../collection/PartitionedSerializedPairBuffer.scala | 4 ++-- 2 files changed, 4 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala b/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala index 499dd97c0656a..8bc4e205bc3c6 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala @@ -59,7 +59,7 @@ private[spark] abstract class BlockObjectWriter(val blockId: BlockId) extends Ou def write(key: Any, value: Any) /** - * Notify the writer that a record worth of bytes has been written with writeBytes. + * Notify the writer that a record worth of bytes has been written with OutputStream#write. */ def recordWritten() @@ -215,12 +215,7 @@ private[spark] class DiskBlockObjectWriter( objOut.writeKey(key) objOut.writeValue(value) - numRecordsWritten += 1 - writeMetrics.incShuffleRecordsWritten(1) - - if (numRecordsWritten % 32 == 0) { - updateBytesWritten() - } + recordWritten() } override def write(b: Int): Unit = throw new UnsupportedOperationException() diff --git a/core/src/main/scala/org/apache/spark/util/collection/PartitionedSerializedPairBuffer.scala b/core/src/main/scala/org/apache/spark/util/collection/PartitionedSerializedPairBuffer.scala index b5ca0c62a04f2..ac9ea6393628f 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/PartitionedSerializedPairBuffer.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/PartitionedSerializedPairBuffer.scala @@ -71,10 +71,10 @@ private[spark] class PartitionedSerializedPairBuffer[K, V]( if (keyStart < 0) { throw new Exception(s"Can't grow buffer beyond ${1 << 31} bytes") } - kvSerializationStream.writeObject[Any](key) + kvSerializationStream.writeKey[Any](key) kvSerializationStream.flush() val valueStart = kvBuffer.size - kvSerializationStream.writeObject[Any](value) + kvSerializationStream.writeValue[Any](value) kvSerializationStream.flush() val valueEnd = kvBuffer.size From 1fd31ba08928f8554f74609f48f4344bd69444e5 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 5 May 2015 18:59:46 -0700 Subject: [PATCH 02/75] [SPARK-6231][SQL/DF] Automatically resolve join condition ambiguity for self-joins. See the comment in join function for more information. Author: Reynold Xin Closes #5919 from rxin/self-join-resolve and squashes the following commits: e2fb0da [Reynold Xin] Updated SQLConf comment. 7233a86 [Reynold Xin] Updated comment. 6be2b4d [Reynold Xin] Removed println 9f6b72f [Reynold Xin] [SPARK-6231][SQL/DF] Automatically resolve ambiguity in join condition for self-joins. --- .../expressions/namedExpressions.scala | 5 ++ .../org/apache/spark/sql/DataFrame.scala | 38 +++++++- .../scala/org/apache/spark/sql/SQLConf.scala | 7 ++ .../apache/spark/sql/DataFrameJoinSuite.scala | 86 +++++++++++++++++++ .../org/apache/spark/sql/DataFrameSuite.scala | 39 --------- .../sql/hive/HiveDataFrameJoinSuite.scala | 38 ++++++++ 6 files changed, 170 insertions(+), 43 deletions(-) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameJoinSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala index afcb2ce8b9cb4..57ace2a14f77c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala @@ -171,6 +171,11 @@ case class AttributeReference( val exprId: ExprId = NamedExpression.newExprId, val qualifiers: Seq[String] = Nil) extends Attribute with trees.LeafNode[Expression] { + /** + * Returns true iff the expression id is the same for both attributes. + */ + def sameRef(other: AttributeReference): Boolean = this.exprId == other.exprId + override def equals(other: Any): Boolean = other match { case ar: AttributeReference => name == ar.name && exprId == ar.exprId && dataType == ar.dataType case _ => false diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala index cf344710ff8b4..aceb7a9627edd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala @@ -416,9 +416,7 @@ class DataFrame private[sql]( * }}} * @group dfops */ - def join(right: DataFrame, joinExprs: Column): DataFrame = { - Join(logicalPlan, right.logicalPlan, joinType = Inner, Some(joinExprs.expr)) - } + def join(right: DataFrame, joinExprs: Column): DataFrame = join(right, joinExprs, "inner") /** * Join with another [[DataFrame]], using the given join expression. The following performs @@ -440,7 +438,39 @@ class DataFrame private[sql]( * @group dfops */ def join(right: DataFrame, joinExprs: Column, joinType: String): DataFrame = { - Join(logicalPlan, right.logicalPlan, JoinType(joinType), Some(joinExprs.expr)) + // Note that in this function, we introduce a hack in the case of self-join to automatically + // resolve ambiguous join conditions into ones that might make sense [SPARK-6231]. + // Consider this case: df.join(df, df("key") === df("key")) + // Since df("key") === df("key") is a trivially true condition, this actually becomes a + // cartesian join. However, most likely users expect to perform a self join using "key". + // With that assumption, this hack turns the trivially true condition into equality on join + // keys that are resolved to both sides. + + // Trigger analysis so in the case of self-join, the analyzer will clone the plan. + // After the cloning, left and right side will have distinct expression ids. + val plan = Join(logicalPlan, right.logicalPlan, JoinType(joinType), Some(joinExprs.expr)) + .queryExecution.analyzed.asInstanceOf[Join] + + // If auto self join alias is disabled, return the plan. + if (!sqlContext.conf.dataFrameSelfJoinAutoResolveAmbiguity) { + return plan + } + + // If left/right have no output set intersection, return the plan. + val lanalyzed = this.logicalPlan.queryExecution.analyzed + val ranalyzed = right.logicalPlan.queryExecution.analyzed + if (lanalyzed.outputSet.intersect(ranalyzed.outputSet).isEmpty) { + return plan + } + + // Otherwise, find the trivially true predicates and automatically resolves them to both sides. + // By the time we get here, since we have already run analysis, all attributes should've been + // resolved and become AttributeReference. + val cond = plan.condition.map { _.transform { + case EqualTo(a: AttributeReference, b: AttributeReference) if a.sameRef(b) => + EqualTo(plan.left.resolve(a.name), plan.right.resolve(b.name)) + }} + plan.copy(condition = cond) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala index 99db959a8741c..3ffc2091d6ba1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala @@ -67,6 +67,10 @@ private[spark] object SQLConf { // Set to false when debugging requires the ability to look at invalid query plans. val DATAFRAME_EAGER_ANALYSIS = "spark.sql.eagerAnalysis" + // Whether to automatically resolve ambiguity in join conditions for self-joins. + // See SPARK-6231. + val DATAFRAME_SELF_JOIN_AUTO_RESOLVE_AMBIGUITY = "spark.sql.selfJoinAutoResolveAmbiguity" + val USE_SQL_SERIALIZER2 = "spark.sql.useSerializer2" object Deprecated { @@ -219,6 +223,9 @@ private[sql] class SQLConf extends Serializable { private[spark] def dataFrameEagerAnalysis: Boolean = getConf(DATAFRAME_EAGER_ANALYSIS, "true").toBoolean + private[spark] def dataFrameSelfJoinAutoResolveAmbiguity: Boolean = + getConf(DATAFRAME_SELF_JOIN_AUTO_RESOLVE_AMBIGUITY, "true").toBoolean + /** ********************** SQLConf functionality methods ************ */ /** Set Spark SQL configuration properties. */ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala new file mode 100644 index 0000000000000..f005f55b6432e --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala @@ -0,0 +1,86 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +import org.apache.spark.sql.TestData._ +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.test.TestSQLContext._ +import org.apache.spark.sql.test.TestSQLContext.implicits._ + + +class DataFrameJoinSuite extends QueryTest { + + test("join - join using") { + val df = Seq(1, 2, 3).map(i => (i, i.toString)).toDF("int", "str") + val df2 = Seq(1, 2, 3).map(i => (i, (i + 1).toString)).toDF("int", "str") + + checkAnswer( + df.join(df2, "int"), + Row(1, "1", "2") :: Row(2, "2", "3") :: Row(3, "3", "4") :: Nil) + } + + test("join - join using self join") { + val df = Seq(1, 2, 3).map(i => (i, i.toString)).toDF("int", "str") + + // self join + checkAnswer( + df.join(df, "int"), + Row(1, "1", "1") :: Row(2, "2", "2") :: Row(3, "3", "3") :: Nil) + } + + test("join - self join") { + val df1 = testData.select(testData("key")).as('df1) + val df2 = testData.select(testData("key")).as('df2) + + checkAnswer( + df1.join(df2, $"df1.key" === $"df2.key"), + sql("SELECT a.key, b.key FROM testData a JOIN testData b ON a.key = b.key").collect().toSeq) + } + + test("join - using aliases after self join") { + val df = Seq(1, 2, 3).map(i => (i, i.toString)).toDF("int", "str") + checkAnswer( + df.as('x).join(df.as('y), $"x.str" === $"y.str").groupBy("x.str").count(), + Row("1", 1) :: Row("2", 1) :: Row("3", 1) :: Nil) + + checkAnswer( + df.as('x).join(df.as('y), $"x.str" === $"y.str").groupBy("y.str").count(), + Row("1", 1) :: Row("2", 1) :: Row("3", 1) :: Nil) + } + + test("[SPARK-6231] join - self join auto resolve ambiguity") { + val df = Seq((1, "1"), (2, "2")).toDF("key", "value") + checkAnswer( + df.join(df, df("key") === df("key")), + Row(1, "1", 1, "1") :: Row(2, "2", 2, "2") :: Nil) + + checkAnswer( + df.join(df.filter($"value" === "2"), df("key") === df("key")), + Row(2, "2", 2, "2") :: Nil) + + checkAnswer( + df.join(df, df("key") === df("key") && df("value") === 1), + Row(1, "1", 1, "1") :: Nil) + + val left = df.groupBy("key").agg($"key", count("*")) + val right = df.groupBy("key").agg($"key", sum("key")) + checkAnswer( + left.join(right, left("key") === right("key")), + Row(1, 1, 1, 1) :: Row(2, 1, 2, 2) :: Nil) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index ff31e15e2d472..1515e9b843771 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -24,7 +24,6 @@ import org.apache.spark.sql.types._ import org.apache.spark.sql.test.{ExamplePointUDT, ExamplePoint, TestSQLContext} import org.apache.spark.sql.test.TestSQLContext.logicalPlanToSparkQuery import org.apache.spark.sql.test.TestSQLContext.implicits._ -import org.apache.spark.sql.test.TestSQLContext.sql class DataFrameSuite extends QueryTest { @@ -118,44 +117,6 @@ class DataFrameSuite extends QueryTest { ) } - test("join - join using") { - val df = Seq(1, 2, 3).map(i => (i, i.toString)).toDF("int", "str") - val df2 = Seq(1, 2, 3).map(i => (i, (i + 1).toString)).toDF("int", "str") - - checkAnswer( - df.join(df2, "int"), - Row(1, "1", "2") :: Row(2, "2", "3") :: Row(3, "3", "4") :: Nil) - } - - test("join - join using self join") { - val df = Seq(1, 2, 3).map(i => (i, i.toString)).toDF("int", "str") - - // self join - checkAnswer( - df.join(df, "int"), - Row(1, "1", "1") :: Row(2, "2", "2") :: Row(3, "3", "3") :: Nil) - } - - test("join - self join") { - val df1 = testData.select(testData("key")).as('df1) - val df2 = testData.select(testData("key")).as('df2) - - checkAnswer( - df1.join(df2, $"df1.key" === $"df2.key"), - sql("SELECT a.key, b.key FROM testData a JOIN testData b ON a.key = b.key").collect().toSeq) - } - - test("join - using aliases after self join") { - val df = Seq(1, 2, 3).map(i => (i, i.toString)).toDF("int", "str") - checkAnswer( - df.as('x).join(df.as('y), $"x.str" === $"y.str").groupBy("x.str").count(), - Row("1", 1) :: Row("2", 1) :: Row("3", 1) :: Nil) - - checkAnswer( - df.as('x).join(df.as('y), $"x.str" === $"y.str").groupBy("y.str").count(), - Row("1", 1) :: Row("2", 1) :: Row("3", 1) :: Nil) - } - test("explode") { val df = Seq((1, "a b c"), (2, "a b"), (3, "a")).toDF("number", "letters") val df2 = diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameJoinSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameJoinSuite.scala new file mode 100644 index 0000000000000..52e782768cb75 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameJoinSuite.scala @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive + +import org.apache.spark.sql.{Row, QueryTest} +import org.apache.spark.sql.hive.test.TestHive.implicits._ + + +class HiveDataFrameJoinSuite extends QueryTest { + + // We should move this into SQL package if we make case sensitivity configurable in SQL. + test("join - self join auto resolve ambiguity with case insensitivity") { + val df = Seq((1, "1"), (2, "2")).toDF("key", "value") + checkAnswer( + df.join(df, df("key") === df("Key")), + Row(1, "1", 1, "1") :: Row(2, "2", 2, "2") :: Nil) + + checkAnswer( + df.join(df.filter($"value" === "2"), df("key") === df("Key")), + Row(2, "2", 2, "2") :: Nil) + } + +} From 51b3d41e160a1326a04536241b427e65b39ed8df Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 5 May 2015 19:27:30 -0700 Subject: [PATCH 03/75] Revert "[SPARK-3454] separate json endpoints for data in the UI" This reverts commit d49735800db27239c11478aac4b0f2ec9df91a3f. The commit broke Spark on Windows. --- .rat-excludes | 7 - core/pom.xml | 8 - .../org/apache/spark/JobExecutionStatus.java | 8 +- .../status/api/v1/ApplicationStatus.java | 30 - .../spark/status/api/v1/StageStatus.java | 31 - .../spark/status/api/v1/TaskSorting.java | 48 - .../java/org/apache/spark/util/EnumUtil.java | 38 - .../scala/org/apache/spark/SparkContext.scala | 2 +- .../org/apache/spark/annotation/Private.java | 41 - .../history/ApplicationHistoryProvider.scala | 4 +- .../deploy/history/FsHistoryProvider.scala | 14 +- .../spark/deploy/history/HistoryServer.scala | 20 +- .../spark/deploy/master/ApplicationInfo.scala | 2 +- .../apache/spark/deploy/master/Master.scala | 14 +- .../deploy/master/ui/ApplicationPage.scala | 19 +- .../spark/deploy/master/ui/MasterPage.scala | 12 +- .../spark/deploy/master/ui/MasterWebUI.scala | 24 +- .../spark/status/api/v1/AllJobsResource.scala | 98 - .../spark/status/api/v1/AllRDDResource.scala | 104 - .../status/api/v1/AllStagesResource.scala | 309 - .../api/v1/ApplicationListResource.scala | 94 - .../status/api/v1/ExecutorListResource.scala | 36 - .../status/api/v1/JacksonMessageWriter.scala | 93 - .../status/api/v1/JsonRootResource.scala | 255 - .../api/v1/OneApplicationResource.scala | 31 - .../spark/status/api/v1/OneJobResource.scala | 41 - .../spark/status/api/v1/OneRDDResource.scala | 34 - .../status/api/v1/OneStageResource.scala | 150 - .../spark/status/api/v1/SecurityFilter.scala | 38 - .../spark/status/api/v1/SimpleDateParam.scala | 55 - .../org/apache/spark/status/api/v1/api.scala | 228 - .../spark/storage/StorageStatusListener.scala | 6 +- .../scala/org/apache/spark/ui/SparkUI.scala | 49 +- .../scala/org/apache/spark/ui/WebUI.scala | 8 +- .../apache/spark/ui/exec/ExecutorsPage.scala | 17 +- .../apache/spark/ui/jobs/AllJobsPage.scala | 14 +- .../apache/spark/ui/jobs/AllStagesPage.scala | 12 +- .../org/apache/spark/ui/jobs/JobPage.scala | 2 +- .../spark/ui/jobs/JobProgressListener.scala | 4 - .../org/apache/spark/ui/jobs/PoolPage.scala | 2 +- .../org/apache/spark/ui/jobs/StagePage.scala | 19 +- .../org/apache/spark/ui/storage/RDDPage.scala | 73 +- .../apache/spark/ui/storage/StoragePage.scala | 2 +- .../apache/spark/ui/storage/StorageTab.scala | 6 +- .../applications/json_expectation | 53 - .../executors/json_expectation | 17 - .../jobs/0/json_expectation | 15 - .../local-1422981780767/jobs/json_expectation | 43 - .../json_expectation | 43 - .../jobs?status=succeeded/json_expectation | 29 - .../local-1422981780767/json_expectation | 10 - .../stages/1/0/json_expectation | 270 - .../stages/1/json_expectation | 270 - .../stages/json_expectation | 89 - .../stages?status=complete/json_expectation | 67 - .../stages?status=failed/json_expectation | 23 - .../storage/rdd/0/json_expectation | 64 - .../storage/rdd/json_expectation | 9 - .../1/jobs/json_expectation | 15 - .../1/stages/0/0/json_expectation | 242 - .../1/stages/0/0/taskList/json_expectation | 193 - .../1/stages/json_expectation | 27 - .../2/jobs/json_expectation | 15 - .../2/stages/0/0/taskList/json_expectation | 193 - .../local-1426533911241/json_expectation | 17 - .../stages/20/0/taskList/json_expectation | 481 - .../json_expectation | 1201 -- .../taskList?sortBy=-runtime/json_expectation | 481 - .../json_expectation | 481 - .../taskList?sortBy=runtime/json_expectation | 481 - .../stages/20/0/taskSummary/json_expectation | 15 - .../json_expectation | 15 - .../json_expectation | 10 - .../json_expectation | 19 - .../json_expectation | 35 - .../json_expectation | 53 - .../json_expectation | 1 - .../local-1422981759269/APPLICATION_COMPLETE | 0 .../local-1422981759269/EVENT_LOG_1 | 88 - .../local-1422981759269/SPARK_VERSION_1.2.0 | 0 .../local-1422981780767/APPLICATION_COMPLETE | 0 .../local-1422981780767/EVENT_LOG_1 | 82 - .../local-1422981780767/SPARK_VERSION_1.2.0 | 0 .../local-1425081759269/APPLICATION_COMPLETE | 0 .../local-1425081759269/EVENT_LOG_1 | 88 - .../local-1425081759269/SPARK_VERSION_1.2.0 | 0 .../local-1426533911241/APPLICATION_COMPLETE | 0 .../local-1426533911241/EVENT_LOG_1 | 24 - .../local-1426533911241/SPARK_VERSION_1.2.0 | 0 .../local-1426633911242/APPLICATION_COMPLETE | 0 .../local-1426633911242/EVENT_LOG_1 | 24 - .../local-1426633911242/SPARK_VERSION_1.2.0 | 0 .../spark-events/local-1427397477963 | 12083 ---------------- .../org/apache/spark/JsonTestUtils.scala | 34 - .../spark/deploy/JsonProtocolSuite.scala | 14 +- .../deploy/history/HistoryServerSuite.scala | 223 +- .../status/api/v1/SimpleDateParamTest.scala | 29 - .../org/apache/spark/ui/UISeleniumSuite.scala | 264 +- docs/monitoring.md | 74 - pom.xml | 12 - 100 files changed, 172 insertions(+), 19946 deletions(-) delete mode 100644 core/src/main/java/org/apache/spark/status/api/v1/ApplicationStatus.java delete mode 100644 core/src/main/java/org/apache/spark/status/api/v1/StageStatus.java delete mode 100644 core/src/main/java/org/apache/spark/status/api/v1/TaskSorting.java delete mode 100644 core/src/main/java/org/apache/spark/util/EnumUtil.java delete mode 100644 core/src/main/scala/org/apache/spark/annotation/Private.java delete mode 100644 core/src/main/scala/org/apache/spark/status/api/v1/AllJobsResource.scala delete mode 100644 core/src/main/scala/org/apache/spark/status/api/v1/AllRDDResource.scala delete mode 100644 core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala delete mode 100644 core/src/main/scala/org/apache/spark/status/api/v1/ApplicationListResource.scala delete mode 100644 core/src/main/scala/org/apache/spark/status/api/v1/ExecutorListResource.scala delete mode 100644 core/src/main/scala/org/apache/spark/status/api/v1/JacksonMessageWriter.scala delete mode 100644 core/src/main/scala/org/apache/spark/status/api/v1/JsonRootResource.scala delete mode 100644 core/src/main/scala/org/apache/spark/status/api/v1/OneApplicationResource.scala delete mode 100644 core/src/main/scala/org/apache/spark/status/api/v1/OneJobResource.scala delete mode 100644 core/src/main/scala/org/apache/spark/status/api/v1/OneRDDResource.scala delete mode 100644 core/src/main/scala/org/apache/spark/status/api/v1/OneStageResource.scala delete mode 100644 core/src/main/scala/org/apache/spark/status/api/v1/SecurityFilter.scala delete mode 100644 core/src/main/scala/org/apache/spark/status/api/v1/SimpleDateParam.scala delete mode 100644 core/src/main/scala/org/apache/spark/status/api/v1/api.scala delete mode 100644 core/src/test/resources/HistoryServerExpectations/applications/json_expectation delete mode 100644 core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/executors/json_expectation delete mode 100644 core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/jobs/0/json_expectation delete mode 100644 core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/jobs/json_expectation delete mode 100644 core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/jobs?status=succeeded&status=failed/json_expectation delete mode 100644 core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/jobs?status=succeeded/json_expectation delete mode 100644 core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/json_expectation delete mode 100644 core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/stages/1/0/json_expectation delete mode 100644 core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/stages/1/json_expectation delete mode 100644 core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/stages/json_expectation delete mode 100644 core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/stages?status=complete/json_expectation delete mode 100644 core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/stages?status=failed/json_expectation delete mode 100644 core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/storage/rdd/0/json_expectation delete mode 100644 core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/storage/rdd/json_expectation delete mode 100644 core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/1/jobs/json_expectation delete mode 100644 core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/1/stages/0/0/json_expectation delete mode 100644 core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/1/stages/0/0/taskList/json_expectation delete mode 100644 core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/1/stages/json_expectation delete mode 100644 core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/2/jobs/json_expectation delete mode 100644 core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/2/stages/0/0/taskList/json_expectation delete mode 100644 core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/json_expectation delete mode 100644 core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskList/json_expectation delete mode 100644 core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskList?offset=10&length=50/json_expectation delete mode 100644 core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskList?sortBy=-runtime/json_expectation delete mode 100644 core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskList?sortBy=DECREASING_RUNTIME/json_expectation delete mode 100644 core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskList?sortBy=runtime/json_expectation delete mode 100644 core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskSummary/json_expectation delete mode 100644 core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskSummary?quantiles=0.01,0.5,0.99/json_expectation delete mode 100644 core/src/test/resources/HistoryServerExpectations/applications?maxDate=2015-02-03T10:42:40.000CST/json_expectation delete mode 100644 core/src/test/resources/HistoryServerExpectations/applications?maxDate=2015-02-10/json_expectation delete mode 100644 core/src/test/resources/HistoryServerExpectations/applications?minDate=2015-02-10/json_expectation delete mode 100644 core/src/test/resources/HistoryServerExpectations/applications?status=completed/json_expectation delete mode 100644 core/src/test/resources/HistoryServerExpectations/applications?status=running/json_expectation delete mode 100755 core/src/test/resources/spark-events/local-1422981759269/APPLICATION_COMPLETE delete mode 100755 core/src/test/resources/spark-events/local-1422981759269/EVENT_LOG_1 delete mode 100755 core/src/test/resources/spark-events/local-1422981759269/SPARK_VERSION_1.2.0 delete mode 100755 core/src/test/resources/spark-events/local-1422981780767/APPLICATION_COMPLETE delete mode 100755 core/src/test/resources/spark-events/local-1422981780767/EVENT_LOG_1 delete mode 100755 core/src/test/resources/spark-events/local-1422981780767/SPARK_VERSION_1.2.0 delete mode 100755 core/src/test/resources/spark-events/local-1425081759269/APPLICATION_COMPLETE delete mode 100755 core/src/test/resources/spark-events/local-1425081759269/EVENT_LOG_1 delete mode 100755 core/src/test/resources/spark-events/local-1425081759269/SPARK_VERSION_1.2.0 delete mode 100755 core/src/test/resources/spark-events/local-1426533911241/APPLICATION_COMPLETE delete mode 100755 core/src/test/resources/spark-events/local-1426533911241/EVENT_LOG_1 delete mode 100755 core/src/test/resources/spark-events/local-1426533911241/SPARK_VERSION_1.2.0 delete mode 100755 core/src/test/resources/spark-events/local-1426633911242/APPLICATION_COMPLETE delete mode 100755 core/src/test/resources/spark-events/local-1426633911242/EVENT_LOG_1 delete mode 100755 core/src/test/resources/spark-events/local-1426633911242/SPARK_VERSION_1.2.0 delete mode 100755 core/src/test/resources/spark-events/local-1427397477963 delete mode 100644 core/src/test/scala/org/apache/spark/JsonTestUtils.scala delete mode 100644 core/src/test/scala/org/apache/spark/status/api/v1/SimpleDateParamTest.scala diff --git a/.rat-excludes b/.rat-excludes index ac652ed29fc09..dccf2db8055ce 100644 --- a/.rat-excludes +++ b/.rat-excludes @@ -74,12 +74,5 @@ logs .*scalastyle-output.xml .*dependency-reduced-pom.xml known_translations -json_expectation -local-1422981759269/* -local-1422981780767/* -local-1425081759269/* -local-1426533911241/* -local-1426633911242/* -local-1427397477963/* DESCRIPTION NAMESPACE diff --git a/core/pom.xml b/core/pom.xml index fc42f48973fe9..164a836cb8f0b 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -228,14 +228,6 @@ json4s-jackson_${scala.binary.version} 3.2.10 - - com.sun.jersey - jersey-server - - - com.sun.jersey - jersey-core - org.apache.mesos mesos diff --git a/core/src/main/java/org/apache/spark/JobExecutionStatus.java b/core/src/main/java/org/apache/spark/JobExecutionStatus.java index 0287fb79f8dd2..6e161313702bb 100644 --- a/core/src/main/java/org/apache/spark/JobExecutionStatus.java +++ b/core/src/main/java/org/apache/spark/JobExecutionStatus.java @@ -17,15 +17,9 @@ package org.apache.spark; -import org.apache.spark.util.EnumUtil; - public enum JobExecutionStatus { RUNNING, SUCCEEDED, FAILED, - UNKNOWN; - - public static JobExecutionStatus fromString(String str) { - return EnumUtil.parseIgnoreCase(JobExecutionStatus.class, str); - } + UNKNOWN } diff --git a/core/src/main/java/org/apache/spark/status/api/v1/ApplicationStatus.java b/core/src/main/java/org/apache/spark/status/api/v1/ApplicationStatus.java deleted file mode 100644 index 8c7dcf776fda8..0000000000000 --- a/core/src/main/java/org/apache/spark/status/api/v1/ApplicationStatus.java +++ /dev/null @@ -1,30 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.status.api.v1; - -import org.apache.spark.util.EnumUtil; - -public enum ApplicationStatus { - COMPLETED, - RUNNING; - - public static ApplicationStatus fromString(String str) { - return EnumUtil.parseIgnoreCase(ApplicationStatus.class, str); - } - -} diff --git a/core/src/main/java/org/apache/spark/status/api/v1/StageStatus.java b/core/src/main/java/org/apache/spark/status/api/v1/StageStatus.java deleted file mode 100644 index 9dbb565aab707..0000000000000 --- a/core/src/main/java/org/apache/spark/status/api/v1/StageStatus.java +++ /dev/null @@ -1,31 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.status.api.v1; - -import org.apache.spark.util.EnumUtil; - -public enum StageStatus { - ACTIVE, - COMPLETE, - FAILED, - PENDING; - - public static StageStatus fromString(String str) { - return EnumUtil.parseIgnoreCase(StageStatus.class, str); - } -} diff --git a/core/src/main/java/org/apache/spark/status/api/v1/TaskSorting.java b/core/src/main/java/org/apache/spark/status/api/v1/TaskSorting.java deleted file mode 100644 index f19ed01d5aebf..0000000000000 --- a/core/src/main/java/org/apache/spark/status/api/v1/TaskSorting.java +++ /dev/null @@ -1,48 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.status.api.v1; - -import org.apache.spark.util.EnumUtil; - -import java.util.HashSet; -import java.util.Set; - -public enum TaskSorting { - ID, - INCREASING_RUNTIME("runtime"), - DECREASING_RUNTIME("-runtime"); - - private final Set alternateNames; - private TaskSorting(String... names) { - alternateNames = new HashSet(); - for (String n: names) { - alternateNames.add(n); - } - } - - public static TaskSorting fromString(String str) { - String lower = str.toLowerCase(); - for (TaskSorting t: values()) { - if (t.alternateNames.contains(lower)) { - return t; - } - } - return EnumUtil.parseIgnoreCase(TaskSorting.class, str); - } - -} diff --git a/core/src/main/java/org/apache/spark/util/EnumUtil.java b/core/src/main/java/org/apache/spark/util/EnumUtil.java deleted file mode 100644 index c40c7e727613c..0000000000000 --- a/core/src/main/java/org/apache/spark/util/EnumUtil.java +++ /dev/null @@ -1,38 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.util; - -import com.google.common.base.Joiner; -import org.apache.spark.annotation.Private; - -@Private -public class EnumUtil { - public static > E parseIgnoreCase(Class clz, String str) { - E[] constants = clz.getEnumConstants(); - if (str == null) { - return null; - } - for (E e : constants) { - if (e.name().equalsIgnoreCase(str)) { - return e; - } - } - throw new IllegalArgumentException( - String.format("Illegal type='%s'. Supported type values: %s", - str, Joiner.on(", ").join(constants))); - } -} diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 2ca6882c8d890..682dec44ac1a5 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -428,7 +428,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli _ui = if (conf.getBoolean("spark.ui.enabled", true)) { Some(SparkUI.createLiveUI(this, _conf, listenerBus, _jobProgressListener, - _env.securityManager,appName, startTime = startTime)) + _env.securityManager,appName)) } else { // For tests, do not enable the UI None diff --git a/core/src/main/scala/org/apache/spark/annotation/Private.java b/core/src/main/scala/org/apache/spark/annotation/Private.java deleted file mode 100644 index 9082fcf0c84bc..0000000000000 --- a/core/src/main/scala/org/apache/spark/annotation/Private.java +++ /dev/null @@ -1,41 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.annotation; - -import java.lang.annotation.ElementType; -import java.lang.annotation.Retention; -import java.lang.annotation.RetentionPolicy; -import java.lang.annotation.Target; - -/** - * A class that is considered private to the internals of Spark -- there is a high-likelihood - * they will be changed in future versions of Spark. - * - * This should be used only when the standard Scala / Java means of protecting classes are - * insufficient. In particular, Java has no equivalent of private[spark], so we use this annotation - * in its place. - * - * NOTE: If there exists a Scaladoc comment that immediately precedes this annotation, the first - * line of the comment must be ":: Private ::" with no trailing blank line. This is because - * of the known issue that Scaladoc displays only either the annotation or the comment, whichever - * comes first. - */ -@Retention(RetentionPolicy.RUNTIME) -@Target({ElementType.TYPE, ElementType.FIELD, ElementType.METHOD, ElementType.PARAMETER, - ElementType.CONSTRUCTOR, ElementType.LOCAL_VARIABLE, ElementType.PACKAGE}) -public @interface Private {} diff --git a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala index 298a8201960d1..6a5011af17458 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala @@ -19,7 +19,7 @@ package org.apache.spark.deploy.history import org.apache.spark.ui.SparkUI -private[spark] case class ApplicationAttemptInfo( +private[history] case class ApplicationAttemptInfo( attemptId: Option[String], startTime: Long, endTime: Long, @@ -27,7 +27,7 @@ private[spark] case class ApplicationAttemptInfo( sparkUser: String, completed: Boolean = false) -private[spark] case class ApplicationHistoryInfo( +private[history] case class ApplicationHistoryInfo( id: String, name: String, attempts: List[ApplicationAttemptInfo]) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index 45c2be34c8680..993763f3aa092 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -17,21 +17,23 @@ package org.apache.spark.deploy.history -import java.io.{BufferedInputStream, FileNotFoundException, IOException, InputStream} +import java.io.{IOException, BufferedInputStream, FileNotFoundException, InputStream} import java.util.concurrent.{ExecutorService, Executors, TimeUnit} import scala.collection.mutable +import scala.concurrent.duration.Duration -import com.google.common.util.concurrent.{MoreExecutors, ThreadFactoryBuilder} -import org.apache.hadoop.fs.{FileStatus, Path} -import org.apache.hadoop.fs.permission.AccessControlException +import com.google.common.util.concurrent.ThreadFactoryBuilder -import org.apache.spark.{Logging, SecurityManager, SparkConf} +import com.google.common.util.concurrent.MoreExecutors +import org.apache.hadoop.fs.permission.AccessControlException +import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.io.CompressionCodec import org.apache.spark.scheduler._ import org.apache.spark.ui.SparkUI import org.apache.spark.util.{Clock, SystemClock, ThreadUtils, Utils} +import org.apache.spark.{Logging, SecurityManager, SparkConf} /** * A class that provides application history from event logs stored in the file system. @@ -149,7 +151,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) val conf = this.conf.clone() val appSecManager = new SecurityManager(conf) SparkUI.createHistoryUI(conf, replayBus, appSecManager, appId, - HistoryServer.getAttemptURI(appId, attempt.attemptId), attempt.startTime) + HistoryServer.getAttemptURI(appId, attempt.attemptId)) // Do not call ui.bind() to avoid creating a new server for each application } diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index 50522e69dc519..754c8e9b6668b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -25,7 +25,6 @@ import org.eclipse.jetty.servlet.{ServletContextHandler, ServletHolder} import org.apache.spark.{Logging, SecurityManager, SparkConf} import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.status.api.v1.{ApplicationInfo, ApplicationsListResource, JsonRootResource, UIRoot} import org.apache.spark.ui.{SparkUI, UIUtils, WebUI} import org.apache.spark.ui.JettyUtils._ import org.apache.spark.util.{SignalLogger, Utils} @@ -46,7 +45,7 @@ class HistoryServer( provider: ApplicationHistoryProvider, securityManager: SecurityManager, port: Int) - extends WebUI(securityManager, port, conf) with Logging with UIRoot { + extends WebUI(securityManager, port, conf) with Logging { // How many applications to retain private val retainedApplications = conf.getInt("spark.history.retainedApplications", 50) @@ -57,7 +56,7 @@ class HistoryServer( require(parts.length == 1 || parts.length == 2, s"Invalid app key $key") val ui = provider .getAppUI(parts(0), if (parts.length > 1) Some(parts(1)) else None) - .getOrElse(throw new NoSuchElementException(s"no app with key $key")) + .getOrElse(throw new NoSuchElementException()) attachSparkUI(ui) ui } @@ -114,10 +113,6 @@ class HistoryServer( } } - def getSparkUI(appKey: String): Option[SparkUI] = { - Option(appCache.get(appKey)) - } - initialize() /** @@ -128,9 +123,6 @@ class HistoryServer( */ def initialize() { attachPage(new HistoryPage(this)) - - attachHandler(JsonRootResource.getJsonServlet(this)) - attachHandler(createStaticHandler(SparkUI.STATIC_RESOURCE_DIR, "/static")) val contextHandler = new ServletContextHandler @@ -168,13 +160,7 @@ class HistoryServer( * * @return List of all known applications. */ - def getApplicationList(): Iterable[ApplicationHistoryInfo] = { - provider.getListing() - } - - def getApplicationInfoList: Iterator[ApplicationInfo] = { - getApplicationList().iterator.map(ApplicationsListResource.appHistoryInfoToPublicAppInfo) - } + def getApplicationList(): Iterable[ApplicationHistoryInfo] = provider.getListing() /** * Returns the provider configuration to show in the listing page. diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala index 1620e95bea218..f59d550d4f3b3 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala @@ -28,7 +28,7 @@ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.deploy.ApplicationDescription import org.apache.spark.util.Utils -private[spark] class ApplicationInfo( +private[deploy] class ApplicationInfo( val startTime: Long, val id: String, val desc: ApplicationDescription, diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 53e1903a3d125..0fac3cdcf55e7 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -754,9 +754,9 @@ private[master] class Master( /** * Rebuild a new SparkUI from the given application's event logs. - * Return the UI if successful, else None + * Return whether this is successful. */ - private[master] def rebuildSparkUI(app: ApplicationInfo): Option[SparkUI] = { + private def rebuildSparkUI(app: ApplicationInfo): Boolean = { val appName = app.desc.name val notFoundBasePath = HistoryServer.UI_PATH_PREFIX + "/not-found" try { @@ -764,7 +764,7 @@ private[master] class Master( .getOrElse { // Event logging is not enabled for this application app.desc.appUiUrl = notFoundBasePath - return None + return false } val eventLogFilePrefix = EventLoggingListener.getLogPath( @@ -787,7 +787,7 @@ private[master] class Master( val logInput = EventLoggingListener.openEventLog(new Path(eventLogFile), fs) val replayBus = new ReplayListenerBus() val ui = SparkUI.createHistoryUI(new SparkConf, replayBus, new SecurityManager(conf), - appName + status, HistoryServer.UI_PATH_PREFIX + s"/${app.id}", app.startTime) + appName + status, HistoryServer.UI_PATH_PREFIX + s"/${app.id}") val maybeTruncated = eventLogFile.endsWith(EventLoggingListener.IN_PROGRESS) try { replayBus.replay(logInput, eventLogFile, maybeTruncated) @@ -798,7 +798,7 @@ private[master] class Master( webUi.attachSparkUI(ui) // Application UI is successfully rebuilt, so link the Master UI to it app.desc.appUiUrl = ui.basePath - Some(ui) + true } catch { case fnf: FileNotFoundException => // Event logging is enabled for this application, but no event logs are found @@ -808,7 +808,7 @@ private[master] class Master( msg += " Did you specify the correct logging directory?" msg = URLEncoder.encode(msg, "UTF-8") app.desc.appUiUrl = notFoundBasePath + s"?msg=$msg&title=$title" - None + false case e: Exception => // Relay exception message to application UI page val title = s"Application history load error (${app.id})" @@ -817,7 +817,7 @@ private[master] class Master( logError(msg, e) msg = URLEncoder.encode(msg, "UTF-8") app.desc.appUiUrl = notFoundBasePath + s"?msg=$msg&exception=$exception&title=$title" - None + false } } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala index 06e265f99e231..273f077bd8f57 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala @@ -23,8 +23,10 @@ import scala.concurrent.Await import scala.xml.Node import akka.pattern.ask +import org.json4s.JValue +import org.json4s.JsonAST.JNothing -import org.apache.spark.deploy.ExecutorState +import org.apache.spark.deploy.{ExecutorState, JsonProtocol} import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState} import org.apache.spark.deploy.master.ExecutorDesc import org.apache.spark.ui.{UIUtils, WebUIPage} @@ -35,6 +37,21 @@ private[ui] class ApplicationPage(parent: MasterWebUI) extends WebUIPage("app") private val master = parent.masterActorRef private val timeout = parent.timeout + /** Executor details for a particular application */ + override def renderJson(request: HttpServletRequest): JValue = { + val appId = request.getParameter("appId") + val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterStateResponse] + val state = Await.result(stateFuture, timeout) + val app = state.activeApps.find(_.id == appId).getOrElse({ + state.completedApps.find(_.id == appId).getOrElse(null) + }) + if (app == null) { + JNothing + } else { + JsonProtocol.writeApplicationInfo(app) + } + } + /** Executor details for a particular application */ def render(request: HttpServletRequest): Seq[Node] = { val appId = request.getParameter("appId") diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala index 756927682cd24..1f2c3fdbfb2bc 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala @@ -35,13 +35,10 @@ private[ui] class MasterPage(parent: MasterWebUI) extends WebUIPage("") { private val master = parent.masterActorRef private val timeout = parent.timeout - def getMasterState: MasterStateResponse = { - val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterStateResponse] - Await.result(stateFuture, timeout) - } - override def renderJson(request: HttpServletRequest): JValue = { - JsonProtocol.writeMasterState(getMasterState) + val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterStateResponse] + val state = Await.result(stateFuture, timeout) + JsonProtocol.writeMasterState(state) } def handleAppKillRequest(request: HttpServletRequest): Unit = { @@ -71,7 +68,8 @@ private[ui] class MasterPage(parent: MasterWebUI) extends WebUIPage("") { /** Index view listing applications and executors */ def render(request: HttpServletRequest): Seq[Node] = { - val state = getMasterState + val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterStateResponse] + val state = Await.result(stateFuture, timeout) val workerHeaders = Seq("Worker Id", "Address", "State", "Cores", "Memory") val workers = state.workers.sortBy(_.id) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala index eb26e9f99c70b..dea0a65eeeaa6 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala @@ -19,7 +19,6 @@ package org.apache.spark.deploy.master.ui import org.apache.spark.Logging import org.apache.spark.deploy.master.Master -import org.apache.spark.status.api.v1.{ApplicationsListResource, ApplicationInfo, JsonRootResource, UIRoot} import org.apache.spark.ui.{SparkUI, WebUI} import org.apache.spark.ui.JettyUtils._ import org.apache.spark.util.RpcUtils @@ -29,15 +28,12 @@ import org.apache.spark.util.RpcUtils */ private[master] class MasterWebUI(val master: Master, requestedPort: Int) - extends WebUI(master.securityMgr, requestedPort, master.conf, name = "MasterUI") with Logging - with UIRoot { + extends WebUI(master.securityMgr, requestedPort, master.conf, name = "MasterUI") with Logging { val masterActorRef = master.self val timeout = RpcUtils.askTimeout(master.conf) val killEnabled = master.conf.getBoolean("spark.ui.killEnabled", true) - val masterPage = new MasterPage(this) - initialize() /** Initialize all components of the server. */ @@ -47,7 +43,6 @@ class MasterWebUI(val master: Master, requestedPort: Int) attachPage(new HistoryNotFoundPage(this)) attachPage(masterPage) attachHandler(createStaticHandler(MasterWebUI.STATIC_RESOURCE_DIR, "/static")) - attachHandler(JsonRootResource.getJsonServlet(this)) attachHandler(createRedirectHandler( "/app/kill", "/", masterPage.handleAppKillRequest, httpMethods = Set("POST"))) attachHandler(createRedirectHandler( @@ -65,23 +60,6 @@ class MasterWebUI(val master: Master, requestedPort: Int) assert(serverInfo.isDefined, "Master UI must be bound to a server before detaching SparkUIs") ui.getHandlers.foreach(detachHandler) } - - def getApplicationInfoList: Iterator[ApplicationInfo] = { - val state = masterPage.getMasterState - val activeApps = state.activeApps.sortBy(_.startTime).reverse - val completedApps = state.completedApps.sortBy(_.endTime).reverse - activeApps.iterator.map { ApplicationsListResource.convertApplicationInfo(_, false) } ++ - completedApps.iterator.map { ApplicationsListResource.convertApplicationInfo(_, true) } - } - - def getSparkUI(appId: String): Option[SparkUI] = { - val state = masterPage.getMasterState - val activeApps = state.activeApps.sortBy(_.startTime).reverse - val completedApps = state.completedApps.sortBy(_.endTime).reverse - (activeApps ++ completedApps).find { _.id == appId }.flatMap { - master.rebuildSparkUI - } - } } private[master] object MasterWebUI { diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/AllJobsResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/AllJobsResource.scala deleted file mode 100644 index 5783df5d8220c..0000000000000 --- a/core/src/main/scala/org/apache/spark/status/api/v1/AllJobsResource.scala +++ /dev/null @@ -1,98 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.status.api.v1 - -import java.util.{Arrays, Date, List => JList} -import javax.ws.rs._ -import javax.ws.rs.core.MediaType - -import org.apache.spark.JobExecutionStatus -import org.apache.spark.ui.SparkUI -import org.apache.spark.ui.jobs.JobProgressListener -import org.apache.spark.ui.jobs.UIData.JobUIData - -@Produces(Array(MediaType.APPLICATION_JSON)) -private[v1] class AllJobsResource(ui: SparkUI) { - - @GET - def jobsList(@QueryParam("status") statuses: JList[JobExecutionStatus]): Seq[JobData] = { - val statusToJobs: Seq[(JobExecutionStatus, Seq[JobUIData])] = - AllJobsResource.getStatusToJobs(ui) - val adjStatuses: JList[JobExecutionStatus] = { - if (statuses.isEmpty) { - Arrays.asList(JobExecutionStatus.values(): _*) - } else { - statuses - } - } - val jobInfos = for { - (status, jobs) <- statusToJobs - job <- jobs if adjStatuses.contains(status) - } yield { - AllJobsResource.convertJobData(job, ui.jobProgressListener, false) - } - jobInfos.sortBy{- _.jobId} - } - -} - -private[v1] object AllJobsResource { - - def getStatusToJobs(ui: SparkUI): Seq[(JobExecutionStatus, Seq[JobUIData])] = { - val statusToJobs = ui.jobProgressListener.synchronized { - Seq( - JobExecutionStatus.RUNNING -> ui.jobProgressListener.activeJobs.values.toSeq, - JobExecutionStatus.SUCCEEDED -> ui.jobProgressListener.completedJobs.toSeq, - JobExecutionStatus.FAILED -> ui.jobProgressListener.failedJobs.reverse.toSeq - ) - } - statusToJobs - } - - def convertJobData( - job: JobUIData, - listener: JobProgressListener, - includeStageDetails: Boolean): JobData = { - listener.synchronized { - val lastStageInfo = listener.stageIdToInfo.get(job.stageIds.max) - val lastStageData = lastStageInfo.flatMap { s => - listener.stageIdToData.get((s.stageId, s.attemptId)) - } - val lastStageName = lastStageInfo.map { _.name }.getOrElse("(Unknown Stage Name)") - val lastStageDescription = lastStageData.flatMap { _.description } - new JobData( - jobId = job.jobId, - name = lastStageName, - description = lastStageDescription, - submissionTime = job.submissionTime.map{new Date(_)}, - completionTime = job.completionTime.map{new Date(_)}, - stageIds = job.stageIds, - jobGroup = job.jobGroup, - status = job.status, - numTasks = job.numTasks, - numActiveTasks = job.numActiveTasks, - numCompletedTasks = job.numCompletedTasks, - numSkippedTasks = job.numCompletedTasks, - numFailedTasks = job.numFailedTasks, - numActiveStages = job.numActiveStages, - numCompletedStages = job.completedStageIndices.size, - numSkippedStages = job.numSkippedStages, - numFailedStages = job.numFailedStages - ) - } - } -} diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/AllRDDResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/AllRDDResource.scala deleted file mode 100644 index 645ede26a0879..0000000000000 --- a/core/src/main/scala/org/apache/spark/status/api/v1/AllRDDResource.scala +++ /dev/null @@ -1,104 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.status.api.v1 - -import javax.ws.rs.{GET, Produces} -import javax.ws.rs.core.MediaType - -import org.apache.spark.storage.{RDDInfo, StorageStatus, StorageUtils} -import org.apache.spark.ui.SparkUI -import org.apache.spark.ui.storage.StorageListener - -@Produces(Array(MediaType.APPLICATION_JSON)) -private[v1] class AllRDDResource(ui: SparkUI) { - - @GET - def rddList(): Seq[RDDStorageInfo] = { - val storageStatusList = ui.storageListener.storageStatusList - val rddInfos = ui.storageListener.rddInfoList - rddInfos.map{rddInfo => - AllRDDResource.getRDDStorageInfo(rddInfo.id, rddInfo, storageStatusList, - includeDetails = false) - } - } - -} - -private[spark] object AllRDDResource { - - def getRDDStorageInfo( - rddId: Int, - listener: StorageListener, - includeDetails: Boolean): Option[RDDStorageInfo] = { - val storageStatusList = listener.storageStatusList - listener.rddInfoList.find { _.id == rddId }.map { rddInfo => - getRDDStorageInfo(rddId, rddInfo, storageStatusList, includeDetails) - } - } - - def getRDDStorageInfo( - rddId: Int, - rddInfo: RDDInfo, - storageStatusList: Seq[StorageStatus], - includeDetails: Boolean): RDDStorageInfo = { - val workers = storageStatusList.map { (rddId, _) } - val blockLocations = StorageUtils.getRddBlockLocations(rddId, storageStatusList) - val blocks = storageStatusList - .flatMap { _.rddBlocksById(rddId) } - .sortWith { _._1.name < _._1.name } - .map { case (blockId, status) => - (blockId, status, blockLocations.get(blockId).getOrElse(Seq[String]("Unknown"))) - } - - val dataDistribution = if (includeDetails) { - Some(storageStatusList.map { status => - new RDDDataDistribution( - address = status.blockManagerId.hostPort, - memoryUsed = status.memUsedByRdd(rddId), - memoryRemaining = status.memRemaining, - diskUsed = status.diskUsedByRdd(rddId) - ) } ) - } else { - None - } - val partitions = if (includeDetails) { - Some(blocks.map { case (id, block, locations) => - new RDDPartitionInfo( - blockName = id.name, - storageLevel = block.storageLevel.description, - memoryUsed = block.memSize, - diskUsed = block.diskSize, - executors = locations - ) - } ) - } else { - None - } - - new RDDStorageInfo( - id = rddId, - name = rddInfo.name, - numPartitions = rddInfo.numPartitions, - numCachedPartitions = rddInfo.numCachedPartitions, - storageLevel = rddInfo.storageLevel.description, - memoryUsed = rddInfo.memSize, - diskUsed = rddInfo.diskSize, - dataDistribution = dataDistribution, - partitions = partitions - ) - } -} diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala deleted file mode 100644 index 50608588f09ae..0000000000000 --- a/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala +++ /dev/null @@ -1,309 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.status.api.v1 - -import java.util.{Arrays, Date, List => JList} -import javax.ws.rs.{GET, PathParam, Produces, QueryParam} -import javax.ws.rs.core.MediaType - -import org.apache.spark.executor.{InputMetrics => InternalInputMetrics, OutputMetrics => InternalOutputMetrics, ShuffleReadMetrics => InternalShuffleReadMetrics, ShuffleWriteMetrics => InternalShuffleWriteMetrics, TaskMetrics => InternalTaskMetrics} -import org.apache.spark.scheduler.{AccumulableInfo => InternalAccumulableInfo, StageInfo} -import org.apache.spark.ui.SparkUI -import org.apache.spark.ui.jobs.UIData.{StageUIData, TaskUIData} -import org.apache.spark.util.Distribution - -@Produces(Array(MediaType.APPLICATION_JSON)) -private[v1] class AllStagesResource(ui: SparkUI) { - - @GET - def stageList(@QueryParam("status") statuses: JList[StageStatus]): Seq[StageData] = { - val listener = ui.jobProgressListener - val stageAndStatus = AllStagesResource.stagesAndStatus(ui) - val adjStatuses = { - if (statuses.isEmpty()) { - Arrays.asList(StageStatus.values(): _*) - } else { - statuses - } - } - for { - (status, stageList) <- stageAndStatus - stageInfo: StageInfo <- stageList if adjStatuses.contains(status) - stageUiData: StageUIData <- listener.synchronized { - listener.stageIdToData.get((stageInfo.stageId, stageInfo.attemptId)) - } - } yield { - AllStagesResource.stageUiToStageData(status, stageInfo, stageUiData, includeDetails = false) - } - } -} - -private[v1] object AllStagesResource { - def stageUiToStageData( - status: StageStatus, - stageInfo: StageInfo, - stageUiData: StageUIData, - includeDetails: Boolean): StageData = { - - val taskData = if (includeDetails) { - Some(stageUiData.taskData.map { case (k, v) => k -> convertTaskData(v) } ) - } else { - None - } - val executorSummary = if (includeDetails) { - Some(stageUiData.executorSummary.map { case (k, summary) => - k -> new ExecutorStageSummary( - taskTime = summary.taskTime, - failedTasks = summary.failedTasks, - succeededTasks = summary.succeededTasks, - inputBytes = summary.inputBytes, - outputBytes = summary.outputBytes, - shuffleRead = summary.shuffleRead, - shuffleWrite = summary.shuffleWrite, - memoryBytesSpilled = summary.memoryBytesSpilled, - diskBytesSpilled = summary.diskBytesSpilled - ) - }) - } else { - None - } - - val accumulableInfo = stageUiData.accumulables.values.map { convertAccumulableInfo }.toSeq - - new StageData( - status = status, - stageId = stageInfo.stageId, - attemptId = stageInfo.attemptId, - numActiveTasks = stageUiData.numActiveTasks, - numCompleteTasks = stageUiData.numCompleteTasks, - numFailedTasks = stageUiData.numFailedTasks, - executorRunTime = stageUiData.executorRunTime, - inputBytes = stageUiData.inputBytes, - inputRecords = stageUiData.inputRecords, - outputBytes = stageUiData.outputBytes, - outputRecords = stageUiData.outputRecords, - shuffleReadBytes = stageUiData.shuffleReadTotalBytes, - shuffleReadRecords = stageUiData.shuffleReadRecords, - shuffleWriteBytes = stageUiData.shuffleWriteBytes, - shuffleWriteRecords = stageUiData.shuffleWriteRecords, - memoryBytesSpilled = stageUiData.memoryBytesSpilled, - diskBytesSpilled = stageUiData.diskBytesSpilled, - schedulingPool = stageUiData.schedulingPool, - name = stageInfo.name, - details = stageInfo.details, - accumulatorUpdates = accumulableInfo, - tasks = taskData, - executorSummary = executorSummary - ) - } - - def stagesAndStatus(ui: SparkUI): Seq[(StageStatus, Seq[StageInfo])] = { - val listener = ui.jobProgressListener - listener.synchronized { - Seq( - StageStatus.ACTIVE -> listener.activeStages.values.toSeq, - StageStatus.COMPLETE -> listener.completedStages.reverse.toSeq, - StageStatus.FAILED -> listener.failedStages.reverse.toSeq, - StageStatus.PENDING -> listener.pendingStages.values.toSeq - ) - } - } - - def convertTaskData(uiData: TaskUIData): TaskData = { - new TaskData( - taskId = uiData.taskInfo.taskId, - index = uiData.taskInfo.index, - attempt = uiData.taskInfo.attempt, - launchTime = new Date(uiData.taskInfo.launchTime), - executorId = uiData.taskInfo.executorId, - host = uiData.taskInfo.host, - taskLocality = uiData.taskInfo.taskLocality.toString(), - speculative = uiData.taskInfo.speculative, - accumulatorUpdates = uiData.taskInfo.accumulables.map { convertAccumulableInfo }, - errorMessage = uiData.errorMessage, - taskMetrics = uiData.taskMetrics.map { convertUiTaskMetrics } - ) - } - - def taskMetricDistributions( - allTaskData: Iterable[TaskUIData], - quantiles: Array[Double]): TaskMetricDistributions = { - - val rawMetrics = allTaskData.flatMap{_.taskMetrics}.toSeq - - def metricQuantiles(f: InternalTaskMetrics => Double): IndexedSeq[Double] = - Distribution(rawMetrics.map { d => f(d) }).get.getQuantiles(quantiles) - - // We need to do a lot of similar munging to nested metrics here. For each one, - // we want (a) extract the values for nested metrics (b) make a distribution for each metric - // (c) shove the distribution into the right field in our return type and (d) only return - // a result if the option is defined for any of the tasks. MetricHelper is a little util - // to make it a little easier to deal w/ all of the nested options. Mostly it lets us just - // implement one "build" method, which just builds the quantiles for each field. - - val inputMetrics: Option[InputMetricDistributions] = - new MetricHelper[InternalInputMetrics, InputMetricDistributions](rawMetrics, quantiles) { - def getSubmetrics(raw: InternalTaskMetrics): Option[InternalInputMetrics] = { - raw.inputMetrics - } - - def build: InputMetricDistributions = new InputMetricDistributions( - bytesRead = submetricQuantiles(_.bytesRead), - recordsRead = submetricQuantiles(_.recordsRead) - ) - }.metricOption - - val outputMetrics: Option[OutputMetricDistributions] = - new MetricHelper[InternalOutputMetrics, OutputMetricDistributions](rawMetrics, quantiles) { - def getSubmetrics(raw:InternalTaskMetrics): Option[InternalOutputMetrics] = { - raw.outputMetrics - } - def build: OutputMetricDistributions = new OutputMetricDistributions( - bytesWritten = submetricQuantiles(_.bytesWritten), - recordsWritten = submetricQuantiles(_.recordsWritten) - ) - }.metricOption - - val shuffleReadMetrics: Option[ShuffleReadMetricDistributions] = - new MetricHelper[InternalShuffleReadMetrics, ShuffleReadMetricDistributions](rawMetrics, - quantiles) { - def getSubmetrics(raw: InternalTaskMetrics): Option[InternalShuffleReadMetrics] = { - raw.shuffleReadMetrics - } - def build: ShuffleReadMetricDistributions = new ShuffleReadMetricDistributions( - readBytes = submetricQuantiles(_.totalBytesRead), - readRecords = submetricQuantiles(_.recordsRead), - remoteBytesRead = submetricQuantiles(_.remoteBytesRead), - remoteBlocksFetched = submetricQuantiles(_.remoteBlocksFetched), - localBlocksFetched = submetricQuantiles(_.localBlocksFetched), - totalBlocksFetched = submetricQuantiles(_.totalBlocksFetched), - fetchWaitTime = submetricQuantiles(_.fetchWaitTime) - ) - }.metricOption - - val shuffleWriteMetrics: Option[ShuffleWriteMetricDistributions] = - new MetricHelper[InternalShuffleWriteMetrics, ShuffleWriteMetricDistributions](rawMetrics, - quantiles) { - def getSubmetrics(raw: InternalTaskMetrics): Option[InternalShuffleWriteMetrics] = { - raw.shuffleWriteMetrics - } - def build: ShuffleWriteMetricDistributions = new ShuffleWriteMetricDistributions( - writeBytes = submetricQuantiles(_.shuffleBytesWritten), - writeRecords = submetricQuantiles(_.shuffleRecordsWritten), - writeTime = submetricQuantiles(_.shuffleWriteTime) - ) - }.metricOption - - new TaskMetricDistributions( - quantiles = quantiles, - executorDeserializeTime = metricQuantiles(_.executorDeserializeTime), - executorRunTime = metricQuantiles(_.executorRunTime), - resultSize = metricQuantiles(_.resultSize), - jvmGcTime = metricQuantiles(_.jvmGCTime), - resultSerializationTime = metricQuantiles(_.resultSerializationTime), - memoryBytesSpilled = metricQuantiles(_.memoryBytesSpilled), - diskBytesSpilled = metricQuantiles(_.diskBytesSpilled), - inputMetrics = inputMetrics, - outputMetrics = outputMetrics, - shuffleReadMetrics = shuffleReadMetrics, - shuffleWriteMetrics = shuffleWriteMetrics - ) - } - - def convertAccumulableInfo(acc: InternalAccumulableInfo): AccumulableInfo = { - new AccumulableInfo(acc.id, acc.name, acc.update, acc.value) - } - - def convertUiTaskMetrics(internal: InternalTaskMetrics): TaskMetrics = { - new TaskMetrics( - executorDeserializeTime = internal.executorDeserializeTime, - executorRunTime = internal.executorRunTime, - resultSize = internal.resultSize, - jvmGcTime = internal.jvmGCTime, - resultSerializationTime = internal.resultSerializationTime, - memoryBytesSpilled = internal.memoryBytesSpilled, - diskBytesSpilled = internal.diskBytesSpilled, - inputMetrics = internal.inputMetrics.map { convertInputMetrics }, - outputMetrics = Option(internal.outputMetrics).flatten.map { convertOutputMetrics }, - shuffleReadMetrics = internal.shuffleReadMetrics.map { convertShuffleReadMetrics }, - shuffleWriteMetrics = internal.shuffleWriteMetrics.map { convertShuffleWriteMetrics } - ) - } - - def convertInputMetrics(internal: InternalInputMetrics): InputMetrics = { - new InputMetrics( - bytesRead = internal.bytesRead, - recordsRead = internal.recordsRead - ) - } - - def convertOutputMetrics(internal: InternalOutputMetrics): OutputMetrics = { - new OutputMetrics( - bytesWritten = internal.bytesWritten, - recordsWritten = internal.recordsWritten - ) - } - - def convertShuffleReadMetrics(internal: InternalShuffleReadMetrics): ShuffleReadMetrics = { - new ShuffleReadMetrics( - remoteBlocksFetched = internal.remoteBlocksFetched, - localBlocksFetched = internal.localBlocksFetched, - fetchWaitTime = internal.fetchWaitTime, - remoteBytesRead = internal.remoteBytesRead, - totalBlocksFetched = internal.totalBlocksFetched, - recordsRead = internal.recordsRead - ) - } - - def convertShuffleWriteMetrics(internal: InternalShuffleWriteMetrics): ShuffleWriteMetrics = { - new ShuffleWriteMetrics( - bytesWritten = internal.shuffleBytesWritten, - writeTime = internal.shuffleWriteTime, - recordsWritten = internal.shuffleRecordsWritten - ) - } -} - -/** - * Helper for getting distributions from nested metric types. Many of the metrics we want are - * contained in options inside TaskMetrics (eg., ShuffleWriteMetrics). This makes it easy to handle - * the options (returning None if the metrics are all empty), and extract the quantiles for each - * metric. After creating an instance, call metricOption to get the result type. - */ -private[v1] abstract class MetricHelper[I,O]( - rawMetrics: Seq[InternalTaskMetrics], - quantiles: Array[Double]) { - - def getSubmetrics(raw: InternalTaskMetrics): Option[I] - - def build: O - - val data: Seq[I] = rawMetrics.flatMap(getSubmetrics) - - /** applies the given function to all input metrics, and returns the quantiles */ - def submetricQuantiles(f: I => Double): IndexedSeq[Double] = { - Distribution(data.map { d => f(d) }).get.getQuantiles(quantiles) - } - - def metricOption: Option[O] = { - if (data.isEmpty) { - None - } else { - Some(build) - } - } -} diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/ApplicationListResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/ApplicationListResource.scala deleted file mode 100644 index 17b521f3e1d41..0000000000000 --- a/core/src/main/scala/org/apache/spark/status/api/v1/ApplicationListResource.scala +++ /dev/null @@ -1,94 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.status.api.v1 - -import java.util.{Arrays, Date, List => JList} -import javax.ws.rs.{DefaultValue, GET, Produces, QueryParam} -import javax.ws.rs.core.MediaType - -import org.apache.spark.deploy.history.ApplicationHistoryInfo -import org.apache.spark.deploy.master.{ApplicationInfo => InternalApplicationInfo} - -@Produces(Array(MediaType.APPLICATION_JSON)) -private[v1] class ApplicationListResource(uiRoot: UIRoot) { - - @GET - def appList( - @QueryParam("status") status: JList[ApplicationStatus], - @DefaultValue("2010-01-01") @QueryParam("minDate") minDate: SimpleDateParam, - @DefaultValue("3000-01-01") @QueryParam("maxDate") maxDate: SimpleDateParam) - : Iterator[ApplicationInfo] = { - val allApps = uiRoot.getApplicationInfoList - val adjStatus = { - if (status.isEmpty) { - Arrays.asList(ApplicationStatus.values(): _*) - } else { - status - } - } - val includeCompleted = adjStatus.contains(ApplicationStatus.COMPLETED) - val includeRunning = adjStatus.contains(ApplicationStatus.RUNNING) - allApps.filter { app => - val anyRunning = app.attempts.exists(!_.completed) - // if any attempt is still running, we consider the app to also still be running - val statusOk = (!anyRunning && includeCompleted) || - (anyRunning && includeRunning) - // keep the app if *any* attempts fall in the right time window - val dateOk = app.attempts.exists { attempt => - attempt.startTime.getTime >= minDate.timestamp && - attempt.startTime.getTime <= maxDate.timestamp - } - statusOk && dateOk - } - } -} - -private[spark] object ApplicationsListResource { - def appHistoryInfoToPublicAppInfo(app: ApplicationHistoryInfo): ApplicationInfo = { - new ApplicationInfo( - id = app.id, - name = app.name, - attempts = app.attempts.map { internalAttemptInfo => - new ApplicationAttemptInfo( - attemptId = internalAttemptInfo.attemptId, - startTime = new Date(internalAttemptInfo.startTime), - endTime = new Date(internalAttemptInfo.endTime), - sparkUser = internalAttemptInfo.sparkUser, - completed = internalAttemptInfo.completed - ) - } - ) - } - - def convertApplicationInfo( - internal: InternalApplicationInfo, - completed: Boolean): ApplicationInfo = { - // standalone application info always has just one attempt - new ApplicationInfo( - id = internal.id, - name = internal.desc.name, - attempts = Seq(new ApplicationAttemptInfo( - attemptId = None, - startTime = new Date(internal.startTime), - endTime = new Date(internal.endTime), - sparkUser = internal.desc.user, - completed = completed - )) - ) - } - -} diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/ExecutorListResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/ExecutorListResource.scala deleted file mode 100644 index 8ad4656b4dada..0000000000000 --- a/core/src/main/scala/org/apache/spark/status/api/v1/ExecutorListResource.scala +++ /dev/null @@ -1,36 +0,0 @@ -/* -* Licensed to the Apache Software Foundation (ASF) under one or more -* contributor license agreements. See the NOTICE file distributed with -* this work for additional information regarding copyright ownership. -* The ASF licenses this file to You under the Apache License, Version 2.0 -* (the "License"); you may not use this file except in compliance with -* the License. You may obtain a copy of the License at -* -* http://www.apache.org/licenses/LICENSE-2.0 -* -* Unless required by applicable law or agreed to in writing, software -* distributed under the License is distributed on an "AS IS" BASIS, -* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -* See the License for the specific language governing permissions and -* limitations under the License. -*/ -package org.apache.spark.status.api.v1 - -import javax.ws.rs.{GET, PathParam, Produces} -import javax.ws.rs.core.MediaType - -import org.apache.spark.ui.SparkUI -import org.apache.spark.ui.exec.ExecutorsPage - -@Produces(Array(MediaType.APPLICATION_JSON)) -private[v1] class ExecutorListResource(ui: SparkUI) { - - @GET - def executorList(): Seq[ExecutorSummary] = { - val listener = ui.executorsListener - val storageStatusList = listener.storageStatusList - (0 until storageStatusList.size).map { statusId => - ExecutorsPage.getExecInfo(listener, statusId) - } - } -} diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/JacksonMessageWriter.scala b/core/src/main/scala/org/apache/spark/status/api/v1/JacksonMessageWriter.scala deleted file mode 100644 index 202a5191ad57d..0000000000000 --- a/core/src/main/scala/org/apache/spark/status/api/v1/JacksonMessageWriter.scala +++ /dev/null @@ -1,93 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.status.api.v1 - -import java.io.OutputStream -import java.lang.annotation.Annotation -import java.lang.reflect.Type -import java.text.SimpleDateFormat -import java.util.{Calendar, SimpleTimeZone} -import javax.ws.rs.Produces -import javax.ws.rs.core.{MediaType, MultivaluedMap} -import javax.ws.rs.ext.{MessageBodyWriter, Provider} - -import com.fasterxml.jackson.annotation.JsonInclude -import com.fasterxml.jackson.databind.{ObjectMapper, SerializationFeature} - -/** - * This class converts the POJO metric responses into json, using jackson. - * - * This doesn't follow the standard jersey-jackson plugin options, because we want to stick - * with an old version of jersey (since we have it from yarn anyway) and don't want to pull in lots - * of dependencies from a new plugin. - * - * Note that jersey automatically discovers this class based on its package and its annotations. - */ -@Provider -@Produces(Array(MediaType.APPLICATION_JSON)) -private[v1] class JacksonMessageWriter extends MessageBodyWriter[Object]{ - - val mapper = new ObjectMapper() { - override def writeValueAsString(t: Any): String = { - super.writeValueAsString(t) - } - } - mapper.registerModule(com.fasterxml.jackson.module.scala.DefaultScalaModule) - mapper.enable(SerializationFeature.INDENT_OUTPUT) - mapper.setSerializationInclusion(JsonInclude.Include.NON_NULL) - mapper.setDateFormat(JacksonMessageWriter.makeISODateFormat) - - override def isWriteable( - aClass: Class[_], - `type`: Type, - annotations: Array[Annotation], - mediaType: MediaType): Boolean = { - true - } - - override def writeTo( - t: Object, - aClass: Class[_], - `type`: Type, - annotations: Array[Annotation], - mediaType: MediaType, - multivaluedMap: MultivaluedMap[String, AnyRef], - outputStream: OutputStream): Unit = { - t match { - case ErrorWrapper(err) => outputStream.write(err.getBytes("utf-8")) - case _ => mapper.writeValue(outputStream, t) - } - } - - override def getSize( - t: Object, - aClass: Class[_], - `type`: Type, - annotations: Array[Annotation], - mediaType: MediaType): Long = { - -1L - } -} - -private[spark] object JacksonMessageWriter { - def makeISODateFormat: SimpleDateFormat = { - val iso8601 = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSS'GMT'") - val cal = Calendar.getInstance(new SimpleTimeZone(0, "GMT")) - iso8601.setCalendar(cal) - iso8601 - } -} diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/JsonRootResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/JsonRootResource.scala deleted file mode 100644 index c3ec45f54681b..0000000000000 --- a/core/src/main/scala/org/apache/spark/status/api/v1/JsonRootResource.scala +++ /dev/null @@ -1,255 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.status.api.v1 - -import javax.servlet.ServletContext -import javax.ws.rs._ -import javax.ws.rs.core.{Context, Response} - -import com.sun.jersey.api.core.ResourceConfig -import com.sun.jersey.spi.container.servlet.ServletContainer -import org.eclipse.jetty.server.handler.ContextHandler -import org.eclipse.jetty.servlet.{ServletContextHandler, ServletHolder} - -import org.apache.spark.SecurityManager -import org.apache.spark.ui.SparkUI - -/** - * Main entry point for serving spark application metrics as json, using JAX-RS. - * - * Each resource should have endpoints that return **public** classes defined in api.scala. Mima - * binary compatibility checks ensure that we don't inadvertently make changes that break the api. - * The returned objects are automatically converted to json by jackson with JacksonMessageWriter. - * In addition, there are a number of tests in HistoryServerSuite that compare the json to "golden - * files". Any changes and additions should be reflected there as well -- see the notes in - * HistoryServerSuite. - */ -@Path("/v1") -private[v1] class JsonRootResource extends UIRootFromServletContext { - - @Path("applications") - def getApplicationList(): ApplicationListResource = { - new ApplicationListResource(uiRoot) - } - - @Path("applications/{appId}") - def getApplication(): OneApplicationResource = { - new OneApplicationResource(uiRoot) - } - - @Path("applications/{appId}/{attemptId}/jobs") - def getJobs( - @PathParam("appId") appId: String, - @PathParam("attemptId") attemptId: String): AllJobsResource = { - uiRoot.withSparkUI(appId, Some(attemptId)) { ui => - new AllJobsResource(ui) - } - } - - @Path("applications/{appId}/jobs") - def getJobs(@PathParam("appId") appId: String): AllJobsResource = { - uiRoot.withSparkUI(appId, None) { ui => - new AllJobsResource(ui) - } - } - - @Path("applications/{appId}/jobs/{jobId: \\d+}") - def getJob(@PathParam("appId") appId: String): OneJobResource = { - uiRoot.withSparkUI(appId, None) { ui => - new OneJobResource(ui) - } - } - - @Path("applications/{appId}/{attemptId}/jobs/{jobId: \\d+}") - def getJob( - @PathParam("appId") appId: String, - @PathParam("attemptId") attemptId: String): OneJobResource = { - uiRoot.withSparkUI(appId, Some(attemptId)) { ui => - new OneJobResource(ui) - } - } - - @Path("applications/{appId}/executors") - def getExecutors(@PathParam("appId") appId: String): ExecutorListResource = { - uiRoot.withSparkUI(appId, None) { ui => - new ExecutorListResource(ui) - } - } - - @Path("applications/{appId}/{attemptId}/executors") - def getExecutors( - @PathParam("appId") appId: String, - @PathParam("attemptId") attemptId: String): ExecutorListResource = { - uiRoot.withSparkUI(appId, Some(attemptId)) { ui => - new ExecutorListResource(ui) - } - } - - - @Path("applications/{appId}/stages") - def getStages(@PathParam("appId") appId: String): AllStagesResource= { - uiRoot.withSparkUI(appId, None) { ui => - new AllStagesResource(ui) - } - } - - @Path("applications/{appId}/{attemptId}/stages") - def getStages( - @PathParam("appId") appId: String, - @PathParam("attemptId") attemptId: String): AllStagesResource= { - uiRoot.withSparkUI(appId, Some(attemptId)) { ui => - new AllStagesResource(ui) - } - } - - @Path("applications/{appId}/stages/{stageId: \\d+}") - def getStage(@PathParam("appId") appId: String): OneStageResource= { - uiRoot.withSparkUI(appId, None) { ui => - new OneStageResource(ui) - } - } - - @Path("applications/{appId}/{attemptId}/stages/{stageId: \\d+}") - def getStage( - @PathParam("appId") appId: String, - @PathParam("attemptId") attemptId: String): OneStageResource = { - uiRoot.withSparkUI(appId, Some(attemptId)) { ui => - new OneStageResource(ui) - } - } - - @Path("applications/{appId}/storage/rdd") - def getRdds(@PathParam("appId") appId: String): AllRDDResource = { - uiRoot.withSparkUI(appId, None) { ui => - new AllRDDResource(ui) - } - } - - @Path("applications/{appId}/{attemptId}/storage/rdd") - def getRdds( - @PathParam("appId") appId: String, - @PathParam("attemptId") attemptId: String): AllRDDResource = { - uiRoot.withSparkUI(appId, Some(attemptId)) { ui => - new AllRDDResource(ui) - } - } - - @Path("applications/{appId}/storage/rdd/{rddId: \\d+}") - def getRdd(@PathParam("appId") appId: String): OneRDDResource = { - uiRoot.withSparkUI(appId, None) { ui => - new OneRDDResource(ui) - } - } - - @Path("applications/{appId}/{attemptId}/storage/rdd/{rddId: \\d+}") - def getRdd( - @PathParam("appId") appId: String, - @PathParam("attemptId") attemptId: String): OneRDDResource = { - uiRoot.withSparkUI(appId, Some(attemptId)) { ui => - new OneRDDResource(ui) - } - } - -} - -private[spark] object JsonRootResource { - - def getJsonServlet(uiRoot: UIRoot): ServletContextHandler = { - val jerseyContext = new ServletContextHandler(ServletContextHandler.NO_SESSIONS) - jerseyContext.setContextPath("/json") - val holder:ServletHolder = new ServletHolder(classOf[ServletContainer]) - holder.setInitParameter("com.sun.jersey.config.property.resourceConfigClass", - "com.sun.jersey.api.core.PackagesResourceConfig") - holder.setInitParameter("com.sun.jersey.config.property.packages", - "org.apache.spark.status.api.v1") - holder.setInitParameter(ResourceConfig.PROPERTY_CONTAINER_REQUEST_FILTERS, - classOf[SecurityFilter].getCanonicalName) - UIRootFromServletContext.setUiRoot(jerseyContext, uiRoot) - jerseyContext.addServlet(holder, "/*") - jerseyContext - } -} - -/** - * This trait is shared by the all the root containers for application UI information -- - * the HistoryServer, the Master UI, and the application UI. This provides the common - * interface needed for them all to expose application info as json. - */ -private[spark] trait UIRoot { - def getSparkUI(appKey: String): Option[SparkUI] - def getApplicationInfoList: Iterator[ApplicationInfo] - - /** - * Get the spark UI with the given appID, and apply a function - * to it. If there is no such app, throw an appropriate exception - */ - def withSparkUI[T](appId: String, attemptId: Option[String])(f: SparkUI => T): T = { - val appKey = attemptId.map(appId + "/" + _).getOrElse(appId) - getSparkUI(appKey) match { - case Some(ui) => - f(ui) - case None => throw new NotFoundException("no such app: " + appId) - } - } - def securityManager: SecurityManager -} - -private[v1] object UIRootFromServletContext { - - private val attribute = getClass.getCanonicalName - - def setUiRoot(contextHandler: ContextHandler, uiRoot: UIRoot): Unit = { - contextHandler.setAttribute(attribute, uiRoot) - } - - def getUiRoot(context: ServletContext): UIRoot = { - context.getAttribute(attribute).asInstanceOf[UIRoot] - } -} - -private[v1] trait UIRootFromServletContext { - @Context - var servletContext: ServletContext = _ - - def uiRoot: UIRoot = UIRootFromServletContext.getUiRoot(servletContext) -} - -private[v1] class NotFoundException(msg: String) extends WebApplicationException( - new NoSuchElementException(msg), - Response - .status(Response.Status.NOT_FOUND) - .entity(ErrorWrapper(msg)) - .build() -) - -private[v1] class BadParameterException(msg: String) extends WebApplicationException( - new IllegalArgumentException(msg), - Response - .status(Response.Status.BAD_REQUEST) - .entity(ErrorWrapper(msg)) - .build() -) { - def this(param: String, exp: String, actual: String) = { - this(raw"""Bad value for parameter "$param". Expected a $exp, got "$actual"""") - } -} - -/** - * Signal to JacksonMessageWriter to not convert the message into json (which would result in an - * extra set of quotes). - */ -private[v1] case class ErrorWrapper(s: String) diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/OneApplicationResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/OneApplicationResource.scala deleted file mode 100644 index b5ef72649e295..0000000000000 --- a/core/src/main/scala/org/apache/spark/status/api/v1/OneApplicationResource.scala +++ /dev/null @@ -1,31 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.status.api.v1 - -import javax.ws.rs.core.MediaType -import javax.ws.rs.{Produces, PathParam, GET} - -@Produces(Array(MediaType.APPLICATION_JSON)) -private[v1] class OneApplicationResource(uiRoot: UIRoot) { - - @GET - def getApp(@PathParam("appId") appId: String): ApplicationInfo = { - val apps = uiRoot.getApplicationInfoList.find { _.id == appId } - apps.getOrElse(throw new NotFoundException("unknown app: " + appId)) - } - -} diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/OneJobResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/OneJobResource.scala deleted file mode 100644 index 6d8a60d480aed..0000000000000 --- a/core/src/main/scala/org/apache/spark/status/api/v1/OneJobResource.scala +++ /dev/null @@ -1,41 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.status.api.v1 - -import javax.ws.rs.{PathParam, GET, Produces} -import javax.ws.rs.core.MediaType - -import org.apache.spark.JobExecutionStatus -import org.apache.spark.ui.SparkUI -import org.apache.spark.ui.jobs.UIData.JobUIData - -@Produces(Array(MediaType.APPLICATION_JSON)) -private[v1] class OneJobResource(ui: SparkUI) { - - @GET - def oneJob(@PathParam("jobId") jobId: Int): JobData = { - val statusToJobs: Seq[(JobExecutionStatus, Seq[JobUIData])] = - AllJobsResource.getStatusToJobs(ui) - val jobOpt = statusToJobs.map {_._2} .flatten.find { jobInfo => jobInfo.jobId == jobId} - jobOpt.map { job => - AllJobsResource.convertJobData(job, ui.jobProgressListener, false) - }.getOrElse { - throw new NotFoundException("unknown job: " + jobId) - } - } - -} diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/OneRDDResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/OneRDDResource.scala deleted file mode 100644 index 07b224fac4786..0000000000000 --- a/core/src/main/scala/org/apache/spark/status/api/v1/OneRDDResource.scala +++ /dev/null @@ -1,34 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.status.api.v1 - -import javax.ws.rs.{PathParam, GET, Produces} -import javax.ws.rs.core.MediaType - -import org.apache.spark.ui.SparkUI - -@Produces(Array(MediaType.APPLICATION_JSON)) -private[v1] class OneRDDResource(ui: SparkUI) { - - @GET - def rddData(@PathParam("rddId") rddId: Int): RDDStorageInfo = { - AllRDDResource.getRDDStorageInfo(rddId, ui.storageListener, true).getOrElse( - throw new NotFoundException(s"no rdd found w/ id $rddId") - ) - } - -} diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/OneStageResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/OneStageResource.scala deleted file mode 100644 index fd24aea63a8a1..0000000000000 --- a/core/src/main/scala/org/apache/spark/status/api/v1/OneStageResource.scala +++ /dev/null @@ -1,150 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.status.api.v1 - -import javax.ws.rs._ -import javax.ws.rs.core.MediaType - -import org.apache.spark.SparkException -import org.apache.spark.scheduler.StageInfo -import org.apache.spark.status.api.v1.StageStatus._ -import org.apache.spark.status.api.v1.TaskSorting._ -import org.apache.spark.ui.SparkUI -import org.apache.spark.ui.jobs.JobProgressListener -import org.apache.spark.ui.jobs.UIData.StageUIData - -@Produces(Array(MediaType.APPLICATION_JSON)) -private[v1] class OneStageResource(ui: SparkUI) { - - @GET - @Path("") - def stageData(@PathParam("stageId") stageId: Int): Seq[StageData] = { - withStage(stageId){ stageAttempts => - stageAttempts.map { stage => - AllStagesResource.stageUiToStageData(stage.status, stage.info, stage.ui, - includeDetails = true) - } - } - } - - @GET - @Path("/{stageAttemptId: \\d+}") - def oneAttemptData( - @PathParam("stageId") stageId: Int, - @PathParam("stageAttemptId") stageAttemptId: Int): StageData = { - withStageAttempt(stageId, stageAttemptId) { stage => - AllStagesResource.stageUiToStageData(stage.status, stage.info, stage.ui, - includeDetails = true) - } - } - - @GET - @Path("/{stageAttemptId: \\d+}/taskSummary") - def taskSummary( - @PathParam("stageId") stageId: Int, - @PathParam("stageAttemptId") stageAttemptId: Int, - @DefaultValue("0.05,0.25,0.5,0.75,0.95") @QueryParam("quantiles") quantileString: String) - : TaskMetricDistributions = { - withStageAttempt(stageId, stageAttemptId) { stage => - val quantiles = quantileString.split(",").map { s => - try { - s.toDouble - } catch { - case nfe: NumberFormatException => - throw new BadParameterException("quantiles", "double", s) - } - } - AllStagesResource.taskMetricDistributions(stage.ui.taskData.values, quantiles) - } - } - - @GET - @Path("/{stageAttemptId: \\d+}/taskList") - def taskList( - @PathParam("stageId") stageId: Int, - @PathParam("stageAttemptId") stageAttemptId: Int, - @DefaultValue("0") @QueryParam("offset") offset: Int, - @DefaultValue("20") @QueryParam("length") length: Int, - @DefaultValue("ID") @QueryParam("sortBy") sortBy: TaskSorting): Seq[TaskData] = { - withStageAttempt(stageId, stageAttemptId) { stage => - val tasks = stage.ui.taskData.values.map{AllStagesResource.convertTaskData}.toIndexedSeq - .sorted(OneStageResource.ordering(sortBy)) - tasks.slice(offset, offset + length) - } - } - - private case class StageStatusInfoUi(status: StageStatus, info: StageInfo, ui: StageUIData) - - private def withStage[T](stageId: Int)(f: Seq[StageStatusInfoUi] => T): T = { - val stageAttempts = findStageStatusUIData(ui.jobProgressListener, stageId) - if (stageAttempts.isEmpty) { - throw new NotFoundException("unknown stage: " + stageId) - } else { - f(stageAttempts) - } - } - - private def findStageStatusUIData( - listener: JobProgressListener, - stageId: Int): Seq[StageStatusInfoUi] = { - listener.synchronized { - def getStatusInfoUi(status: StageStatus, infos: Seq[StageInfo]): Seq[StageStatusInfoUi] = { - infos.filter { _.stageId == stageId }.map { info => - val ui = listener.stageIdToData.getOrElse((info.stageId, info.attemptId), - // this is an internal error -- we should always have uiData - throw new SparkException( - s"no stage ui data found for stage: ${info.stageId}:${info.attemptId}") - ) - StageStatusInfoUi(status, info, ui) - } - } - getStatusInfoUi(ACTIVE, listener.activeStages.values.toSeq) ++ - getStatusInfoUi(COMPLETE, listener.completedStages) ++ - getStatusInfoUi(FAILED, listener.failedStages) ++ - getStatusInfoUi(PENDING, listener.pendingStages.values.toSeq) - } - } - - private def withStageAttempt[T]( - stageId: Int, - stageAttemptId: Int) - (f: StageStatusInfoUi => T): T = { - withStage(stageId) { attempts => - val oneAttempt = attempts.find { stage => stage.info.attemptId == stageAttemptId } - oneAttempt match { - case Some(stage) => - f(stage) - case None => - val stageAttempts = attempts.map { _.info.attemptId } - throw new NotFoundException(s"unknown attempt for stage $stageId. " + - s"Found attempts: ${stageAttempts.mkString("[", ",", "]")}") - } - } - } -} - -object OneStageResource { - def ordering(taskSorting: TaskSorting): Ordering[TaskData] = { - val extractor: (TaskData => Long) = td => - taskSorting match { - case ID => td.taskId - case INCREASING_RUNTIME => td.taskMetrics.map{_.executorRunTime}.getOrElse(-1L) - case DECREASING_RUNTIME => -td.taskMetrics.map{_.executorRunTime}.getOrElse(-1L) - } - Ordering.by(extractor) - } -} diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/SecurityFilter.scala b/core/src/main/scala/org/apache/spark/status/api/v1/SecurityFilter.scala deleted file mode 100644 index 95fbd96ade5ab..0000000000000 --- a/core/src/main/scala/org/apache/spark/status/api/v1/SecurityFilter.scala +++ /dev/null @@ -1,38 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.status.api.v1 - -import javax.ws.rs.WebApplicationException -import javax.ws.rs.core.Response - -import com.sun.jersey.spi.container.{ContainerRequest, ContainerRequestFilter} - -private[v1] class SecurityFilter extends ContainerRequestFilter with UIRootFromServletContext { - def filter(req: ContainerRequest): ContainerRequest = { - val user = Option(req.getUserPrincipal).map { _.getName }.orNull - if (uiRoot.securityManager.checkUIViewPermissions(user)) { - req - } else { - throw new WebApplicationException( - Response - .status(Response.Status.FORBIDDEN) - .entity(raw"""user "$user"is not authorized""") - .build() - ) - } - } -} diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/SimpleDateParam.scala b/core/src/main/scala/org/apache/spark/status/api/v1/SimpleDateParam.scala deleted file mode 100644 index cee29786c3019..0000000000000 --- a/core/src/main/scala/org/apache/spark/status/api/v1/SimpleDateParam.scala +++ /dev/null @@ -1,55 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.status.api.v1 - -import java.text.SimpleDateFormat -import java.util.TimeZone -import javax.ws.rs.WebApplicationException -import javax.ws.rs.core.Response -import javax.ws.rs.core.Response.Status - -import scala.util.Try - -private[v1] class SimpleDateParam(val originalValue: String) { - val timestamp: Long = { - SimpleDateParam.formats.collectFirst { - case fmt if Try(fmt.parse(originalValue)).isSuccess => - fmt.parse(originalValue).getTime() - }.getOrElse( - throw new WebApplicationException( - Response - .status(Status.BAD_REQUEST) - .entity("Couldn't parse date: " + originalValue) - .build() - ) - ) - } -} - -private[v1] object SimpleDateParam { - - val formats: Seq[SimpleDateFormat] = { - - val gmtDay = new SimpleDateFormat("yyyy-MM-dd") - gmtDay.setTimeZone(TimeZone.getTimeZone("GMT")) - - Seq( - new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSz"), - gmtDay - ) - } -} 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 deleted file mode 100644 index ef3c8570d8186..0000000000000 --- a/core/src/main/scala/org/apache/spark/status/api/v1/api.scala +++ /dev/null @@ -1,228 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.status.api.v1 - -import java.util.Date - -import scala.collection.Map - -import org.apache.spark.JobExecutionStatus - -class ApplicationInfo private[spark]( - val id: String, - val name: String, - val attempts: Seq[ApplicationAttemptInfo]) - -class ApplicationAttemptInfo private[spark]( - val attemptId: Option[String], - val startTime: Date, - val endTime: Date, - val sparkUser: String, - val completed: Boolean = false) - -class ExecutorStageSummary private[spark]( - val taskTime : Long, - val failedTasks : Int, - val succeededTasks : Int, - val inputBytes : Long, - val outputBytes : Long, - val shuffleRead : Long, - val shuffleWrite : Long, - val memoryBytesSpilled : Long, - val diskBytesSpilled : Long) - -class ExecutorSummary private[spark]( - val id: String, - val hostPort: String, - val rddBlocks: Int, - val memoryUsed: Long, - val diskUsed: Long, - val activeTasks: Int, - val failedTasks: Int, - val completedTasks: Int, - val totalTasks: Int, - val totalDuration: Long, - val totalInputBytes: Long, - val totalShuffleRead: Long, - val totalShuffleWrite: Long, - val maxMemory: Long, - val executorLogs: Map[String, String]) - -class JobData private[spark]( - val jobId: Int, - val name: String, - val description: Option[String], - val submissionTime: Option[Date], - val completionTime: Option[Date], - val stageIds: Seq[Int], - val jobGroup: Option[String], - val status: JobExecutionStatus, - val numTasks: Int, - val numActiveTasks: Int, - val numCompletedTasks: Int, - val numSkippedTasks: Int, - val numFailedTasks: Int, - val numActiveStages: Int, - val numCompletedStages: Int, - val numSkippedStages: Int, - val numFailedStages: Int) - -// Q: should Tachyon size go in here as well? currently the UI only shows it on the overall storage -// page ... does anybody pay attention to it? -class RDDStorageInfo private[spark]( - val id: Int, - val name: String, - val numPartitions: Int, - val numCachedPartitions: Int, - val storageLevel: String, - val memoryUsed: Long, - val diskUsed: Long, - val dataDistribution: Option[Seq[RDDDataDistribution]], - val partitions: Option[Seq[RDDPartitionInfo]]) - -class RDDDataDistribution private[spark]( - val address: String, - val memoryUsed: Long, - val memoryRemaining: Long, - val diskUsed: Long) - -class RDDPartitionInfo private[spark]( - val blockName: String, - val storageLevel: String, - val memoryUsed: Long, - val diskUsed: Long, - val executors: Seq[String]) - -class StageData private[spark]( - val status: StageStatus, - val stageId: Int, - val attemptId: Int, - val numActiveTasks: Int , - val numCompleteTasks: Int, - val numFailedTasks: Int, - - val executorRunTime: Long, - - val inputBytes: Long, - val inputRecords: Long, - val outputBytes: Long, - val outputRecords: Long, - val shuffleReadBytes: Long, - val shuffleReadRecords: Long, - val shuffleWriteBytes: Long, - val shuffleWriteRecords: Long, - val memoryBytesSpilled: Long, - val diskBytesSpilled: Long, - - val name: String, - val details: String, - val schedulingPool: String, - - val accumulatorUpdates: Seq[AccumulableInfo], - val tasks: Option[Map[Long, TaskData]], - val executorSummary:Option[Map[String,ExecutorStageSummary]]) - -class TaskData private[spark]( - val taskId: Long, - val index: Int, - val attempt: Int, - val launchTime: Date, - val executorId: String, - val host: String, - val taskLocality: String, - val speculative: Boolean, - val accumulatorUpdates: Seq[AccumulableInfo], - val errorMessage: Option[String] = None, - val taskMetrics: Option[TaskMetrics] = None) - -class TaskMetrics private[spark]( - val executorDeserializeTime: Long, - val executorRunTime: Long, - val resultSize: Long, - val jvmGcTime: Long, - val resultSerializationTime: Long, - val memoryBytesSpilled: Long, - val diskBytesSpilled: Long, - val inputMetrics: Option[InputMetrics], - val outputMetrics: Option[OutputMetrics], - val shuffleReadMetrics: Option[ShuffleReadMetrics], - val shuffleWriteMetrics: Option[ShuffleWriteMetrics]) - -class InputMetrics private[spark]( - val bytesRead: Long, - val recordsRead: Long) - -class OutputMetrics private[spark]( - val bytesWritten: Long, - val recordsWritten: Long) - -class ShuffleReadMetrics private[spark]( - val remoteBlocksFetched: Int, - val localBlocksFetched: Int, - val fetchWaitTime: Long, - val remoteBytesRead: Long, - val totalBlocksFetched: Int, - val recordsRead: Long) - -class ShuffleWriteMetrics private[spark]( - val bytesWritten: Long, - val writeTime: Long, - val recordsWritten: Long) - -class TaskMetricDistributions private[spark]( - val quantiles: IndexedSeq[Double], - - val executorDeserializeTime: IndexedSeq[Double], - val executorRunTime: IndexedSeq[Double], - val resultSize: IndexedSeq[Double], - val jvmGcTime: IndexedSeq[Double], - val resultSerializationTime: IndexedSeq[Double], - val memoryBytesSpilled: IndexedSeq[Double], - val diskBytesSpilled: IndexedSeq[Double], - - val inputMetrics: Option[InputMetricDistributions], - val outputMetrics: Option[OutputMetricDistributions], - val shuffleReadMetrics: Option[ShuffleReadMetricDistributions], - val shuffleWriteMetrics: Option[ShuffleWriteMetricDistributions]) - -class InputMetricDistributions private[spark]( - val bytesRead: IndexedSeq[Double], - val recordsRead: IndexedSeq[Double]) - -class OutputMetricDistributions private[spark]( - val bytesWritten: IndexedSeq[Double], - val recordsWritten: IndexedSeq[Double]) - -class ShuffleReadMetricDistributions private[spark]( - val readBytes: IndexedSeq[Double], - val readRecords: IndexedSeq[Double], - val remoteBlocksFetched: IndexedSeq[Double], - val localBlocksFetched: IndexedSeq[Double], - val fetchWaitTime: IndexedSeq[Double], - val remoteBytesRead: IndexedSeq[Double], - val totalBlocksFetched: IndexedSeq[Double]) - -class ShuffleWriteMetricDistributions private[spark]( - val writeBytes: IndexedSeq[Double], - val writeRecords: IndexedSeq[Double], - val writeTime: IndexedSeq[Double]) - -class AccumulableInfo private[spark]( - val id: Long, - val name: String, - val update: Option[String], - val value: String) diff --git a/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala b/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala index ec711480ebf30..7d75929b96f75 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala @@ -25,17 +25,13 @@ import org.apache.spark.scheduler._ /** * :: DeveloperApi :: * A SparkListener that maintains executor storage status. - * - * This class is thread-safe (unlike JobProgressListener) */ @DeveloperApi class StorageStatusListener extends SparkListener { // This maintains only blocks that are cached (i.e. storage level is not StorageLevel.NONE) private[storage] val executorIdToStorageStatus = mutable.Map[String, StorageStatus]() - def storageStatusList: Seq[StorageStatus] = synchronized { - executorIdToStorageStatus.values.toSeq - } + def storageStatusList: Seq[StorageStatus] = executorIdToStorageStatus.values.toSeq /** Update storage status list to reflect updated block statuses */ private def updateStorageStatus(execId: String, updatedBlocks: Seq[(BlockId, BlockStatus)]) { diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index bfe4a180e8a6f..a5271f0574e6c 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -17,9 +17,6 @@ package org.apache.spark.ui -import java.util.Date - -import org.apache.spark.status.api.v1.{ApplicationAttemptInfo, ApplicationInfo, JsonRootResource, UIRoot} import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext} import org.apache.spark.scheduler._ import org.apache.spark.storage.StorageStatusListener @@ -36,7 +33,7 @@ import org.apache.spark.ui.scope.RDDOperationGraphListener private[spark] class SparkUI private ( val sc: Option[SparkContext], val conf: SparkConf, - securityManager: SecurityManager, + val securityManager: SecurityManager, val environmentListener: EnvironmentListener, val storageStatusListener: StorageStatusListener, val executorsListener: ExecutorsListener, @@ -44,27 +41,22 @@ private[spark] class SparkUI private ( val storageListener: StorageListener, val operationGraphListener: RDDOperationGraphListener, var appName: String, - val basePath: String, - val startTime: Long) + val basePath: String) extends WebUI(securityManager, SparkUI.getUIPort(conf), conf, basePath, "SparkUI") - with Logging - with UIRoot { + with Logging { val killEnabled = sc.map(_.conf.getBoolean("spark.ui.killEnabled", true)).getOrElse(false) - - val stagesTab = new StagesTab(this) - /** Initialize all components of the server. */ def initialize() { attachTab(new JobsTab(this)) + val stagesTab = new StagesTab(this) attachTab(stagesTab) attachTab(new StorageTab(this)) attachTab(new EnvironmentTab(this)) attachTab(new ExecutorsTab(this)) attachHandler(createStaticHandler(SparkUI.STATIC_RESOURCE_DIR, "/static")) attachHandler(createRedirectHandler("/", "/jobs", basePath = basePath)) - attachHandler(JsonRootResource.getJsonServlet(this)) // This should be POST only, but, the YARN AM proxy won't proxy POSTs attachHandler(createRedirectHandler( "/stages/stage/kill", "/stages", stagesTab.handleKillRequest, @@ -91,24 +83,6 @@ private[spark] class SparkUI private ( private[spark] def appUIHostPort = publicHostName + ":" + boundPort private[spark] def appUIAddress = s"http://$appUIHostPort" - - def getSparkUI(appId: String): Option[SparkUI] = { - if (appId == appName) Some(this) else None - } - - def getApplicationInfoList: Iterator[ApplicationInfo] = { - Iterator(new ApplicationInfo( - id = appName, - name = appName, - attempts = Seq(new ApplicationAttemptInfo( - attemptId = None, - startTime = new Date(startTime), - endTime = new Date(-1), - sparkUser = "", - completed = false - )) - )) - } } private[spark] abstract class SparkUITab(parent: SparkUI, prefix: String) @@ -135,10 +109,9 @@ private[spark] object SparkUI { listenerBus: SparkListenerBus, jobProgressListener: JobProgressListener, securityManager: SecurityManager, - appName: String, - startTime: Long): SparkUI = { + appName: String): SparkUI = { create(Some(sc), conf, listenerBus, securityManager, appName, - jobProgressListener = Some(jobProgressListener), startTime = startTime) + jobProgressListener = Some(jobProgressListener)) } def createHistoryUI( @@ -146,9 +119,8 @@ private[spark] object SparkUI { listenerBus: SparkListenerBus, securityManager: SecurityManager, appName: String, - basePath: String, - startTime: Long): SparkUI = { - create(None, conf, listenerBus, securityManager, appName, basePath, startTime = startTime) + basePath: String): SparkUI = { + create(None, conf, listenerBus, securityManager, appName, basePath) } /** @@ -165,8 +137,7 @@ private[spark] object SparkUI { securityManager: SecurityManager, appName: String, basePath: String = "", - jobProgressListener: Option[JobProgressListener] = None, - startTime: Long): SparkUI = { + jobProgressListener: Option[JobProgressListener] = None): SparkUI = { val _jobProgressListener: JobProgressListener = jobProgressListener.getOrElse { val listener = new JobProgressListener(conf) @@ -188,6 +159,6 @@ private[spark] object SparkUI { new SparkUI(sc, conf, securityManager, environmentListener, storageStatusListener, executorsListener, _jobProgressListener, storageListener, operationGraphListener, - appName, basePath, startTime) + appName, basePath) } } diff --git a/core/src/main/scala/org/apache/spark/ui/WebUI.scala b/core/src/main/scala/org/apache/spark/ui/WebUI.scala index 384f2ad26e281..f9860d1a5ce76 100644 --- a/core/src/main/scala/org/apache/spark/ui/WebUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/WebUI.scala @@ -37,7 +37,7 @@ import org.apache.spark.{Logging, SecurityManager, SparkConf} * pages. The use of tabs is optional, however; a WebUI may choose to include pages directly. */ private[spark] abstract class WebUI( - val securityManager: SecurityManager, + securityManager: SecurityManager, port: Int, conf: SparkConf, basePath: String = "", @@ -77,9 +77,15 @@ private[spark] abstract class WebUI( val pagePath = "/" + page.prefix val renderHandler = createServletHandler(pagePath, (request: HttpServletRequest) => page.render(request), securityManager, basePath) + val renderJsonHandler = createServletHandler(pagePath.stripSuffix("/") + "/json", + (request: HttpServletRequest) => page.renderJson(request), securityManager, basePath) attachHandler(renderHandler) + attachHandler(renderJsonHandler) pageToHandlers.getOrElseUpdate(page, ArrayBuffer[ServletContextHandler]()) .append(renderHandler) + pageToHandlers.getOrElseUpdate(page, ArrayBuffer[ServletContextHandler]()) + .append(renderJsonHandler) + } /** Attach a handler to this UI. */ diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala index b247e4cdc3bd4..956608d7c0cbe 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala @@ -22,11 +22,11 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node -import org.apache.spark.status.api.v1.ExecutorSummary import org.apache.spark.ui.{ToolTips, UIUtils, WebUIPage} import org.apache.spark.util.Utils -// This isn't even used anymore -- but we need to keep it b/c of a MiMa false positive +/** Summary information about an executor to display in the UI. */ +// Needs to be private[ui] because of a false positive MiMa failure. private[ui] case class ExecutorSummaryInfo( id: String, hostPort: String, @@ -44,7 +44,6 @@ private[ui] case class ExecutorSummaryInfo( maxMemory: Long, executorLogs: Map[String, String]) - private[ui] class ExecutorsPage( parent: ExecutorsTab, threadDumpEnabled: Boolean) @@ -56,8 +55,7 @@ private[ui] class ExecutorsPage( val maxMem = storageStatusList.map(_.maxMem).sum val memUsed = storageStatusList.map(_.memUsed).sum val diskUsed = storageStatusList.map(_.diskUsed).sum - val execInfo = for (statusId <- 0 until storageStatusList.size) yield - ExecutorsPage.getExecInfo(listener, statusId) + val execInfo = for (statusId <- 0 until storageStatusList.size) yield getExecInfo(statusId) val execInfoSorted = execInfo.sortBy(_.id) val logsExist = execInfo.filter(_.executorLogs.nonEmpty).nonEmpty @@ -113,7 +111,7 @@ private[ui] class ExecutorsPage( } /** Render an HTML row representing an executor */ - private def execRow(info: ExecutorSummary, logsExist: Boolean): Seq[Node] = { + private def execRow(info: ExecutorSummaryInfo, logsExist: Boolean): Seq[Node] = { val maximumMemory = info.maxMemory val memoryUsed = info.memoryUsed val diskUsed = info.diskUsed @@ -172,11 +170,8 @@ private[ui] class ExecutorsPage( } -} - -private[spark] object ExecutorsPage { /** Represent an executor's info as a map given a storage status index */ - def getExecInfo(listener: ExecutorsListener, statusId: Int): ExecutorSummary = { + private def getExecInfo(statusId: Int): ExecutorSummaryInfo = { val status = listener.storageStatusList(statusId) val execId = status.blockManagerId.executorId val hostPort = status.blockManagerId.hostPort @@ -194,7 +189,7 @@ private[spark] object ExecutorsPage { val totalShuffleWrite = listener.executorToShuffleWrite.getOrElse(execId, 0L) val executorLogs = listener.executorToLogUrls.getOrElse(execId, Map.empty) - new ExecutorSummary( + new ExecutorSummaryInfo( execId, hostPort, rddBlocks, diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala index 09323d1d80ad6..f6abf27db49dd 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala @@ -271,12 +271,6 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { val shouldShowCompletedJobs = completedJobs.nonEmpty val shouldShowFailedJobs = failedJobs.nonEmpty - val completedJobNumStr = if (completedJobs.size == listener.numCompletedJobs) { - s"${completedJobs.size}" - } else { - s"${listener.numCompletedJobs}, only showing ${completedJobs.size}" - } - val summary: NodeSeq =
    @@ -301,9 +295,9 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { } { if (shouldShowCompletedJobs) { -
  • +
  • Completed Jobs: - {completedJobNumStr} + {completedJobs.size}
  • } } @@ -311,7 +305,7 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { if (shouldShowFailedJobs) {
  • Failed Jobs: - {listener.numFailedJobs} + {failedJobs.size}
  • } } @@ -328,7 +322,7 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { activeJobsTable } if (shouldShowCompletedJobs) { - content ++=

    Completed Jobs ({completedJobNumStr})

    ++ + content ++=

    Completed Jobs ({completedJobs.size})

    ++ completedJobsTable } if (shouldShowFailedJobs) { diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala index a37f739ab9c66..236bc8ea92879 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala @@ -64,12 +64,6 @@ private[ui] class AllStagesPage(parent: StagesTab) extends WebUIPage("") { val shouldShowCompletedStages = completedStages.nonEmpty val shouldShowFailedStages = failedStages.nonEmpty - val completedStageNumStr = if (numCompletedStages == completedStages.size) { - s"$numCompletedStages" - } else { - s"$numCompletedStages, only showing ${completedStages.size}" - } - val summary: NodeSeq =
      @@ -104,9 +98,9 @@ private[ui] class AllStagesPage(parent: StagesTab) extends WebUIPage("") { } { if (shouldShowCompletedStages) { -
    • +
    • Completed Stages: - {completedStageNumStr} + {numCompletedStages}
    • } } @@ -138,7 +132,7 @@ private[ui] class AllStagesPage(parent: StagesTab) extends WebUIPage("") { pendingStagesTable.toNodeSeq } if (shouldShowCompletedStages) { - content ++=

      Completed Stages ({completedStageNumStr})

      ++ + content ++=

      Completed Stages ({numCompletedStages})

      ++ completedStagesTable.toNodeSeq } if (shouldShowFailedStages) { diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala index 7163217e1fed0..96cc3d78d0f15 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala @@ -187,7 +187,7 @@ private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") { val jobDataOption = listener.jobIdToData.get(jobId) if (jobDataOption.isEmpty) { val content = -
      +

      No information to display for job {jobId}

      return UIUtils.headerSparkPage( diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index 246e191d64776..8f9aa9fdec819 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -74,8 +74,6 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { // JobProgressListener's retention limits. var numCompletedStages = 0 var numFailedStages = 0 - var numCompletedJobs = 0 - var numFailedJobs = 0 // Misc: val executorIdToBlockManagerId = HashMap[ExecutorId, BlockManagerId]() @@ -219,12 +217,10 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { completedJobs += jobData trimJobsIfNecessary(completedJobs) jobData.status = JobExecutionStatus.SUCCEEDED - numCompletedJobs += 1 case JobFailed(exception) => failedJobs += jobData trimJobsIfNecessary(failedJobs) jobData.status = JobExecutionStatus.FAILED - numFailedJobs += 1 } for (stageId <- jobData.stageIds) { stageIdToActiveJobIds.get(stageId).foreach { jobsUsingStage => diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala index f3e0b38523f32..d725b9d8565ac 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala @@ -21,7 +21,7 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node -import org.apache.spark.scheduler.StageInfo +import org.apache.spark.scheduler.{Schedulable, StageInfo} import org.apache.spark.ui.{WebUIPage, UIUtils} /** Page showing specific pool details */ diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index 89d175b06b947..579310070c76c 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -25,11 +25,11 @@ import scala.xml.{Elem, Node, Unparsed} import org.apache.commons.lang3.StringEscapeUtils import org.apache.spark.executor.TaskMetrics -import org.apache.spark.scheduler.{AccumulableInfo, TaskInfo} import org.apache.spark.ui.{ToolTips, WebUIPage, UIUtils} import org.apache.spark.ui.jobs.UIData._ import org.apache.spark.ui.scope.RDDOperationGraph import org.apache.spark.util.{Utils, Distribution} +import org.apache.spark.scheduler.{AccumulableInfo, TaskInfo} /** Page showing statistics and task list for a given stage */ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { @@ -48,22 +48,14 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { val stageAttemptId = parameterAttempt.toInt val stageDataOption = progressListener.stageIdToData.get((stageId, stageAttemptId)) - val stageHeader = s"Details for Stage $stageId (Attempt $stageAttemptId)" - if (stageDataOption.isEmpty) { - val content = -
      -

      No information to display for Stage {stageId} (Attempt {stageAttemptId})

      -
      - return UIUtils.headerSparkPage(stageHeader, content, parent) - - } - if (stageDataOption.get.taskData.isEmpty) { + if (stageDataOption.isEmpty || stageDataOption.get.taskData.isEmpty) { val content =

      Summary Metrics

      No tasks have started yet

      Tasks

      No tasks have started yet
      - return UIUtils.headerSparkPage(stageHeader, content, parent) + return UIUtils.headerSparkPage( + s"Details for Stage $stageId (Attempt $stageAttemptId)", content, parent) } val stageData = stageDataOption.get @@ -454,7 +446,8 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { maybeAccumulableTable ++

      Tasks

      ++ taskTable - UIUtils.headerSparkPage(stageHeader, content, parent, showVisualization = true) + UIUtils.headerSparkPage( + "Details for Stage %d".format(stageId), content, parent, showVisualization = true) } } diff --git a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala index 05f94a7507f4f..199f731b92bcc 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala @@ -21,8 +21,8 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node -import org.apache.spark.status.api.v1.{AllRDDResource, RDDDataDistribution, RDDPartitionInfo} -import org.apache.spark.ui.{UIUtils, WebUIPage} +import org.apache.spark.storage.{BlockId, BlockStatus, StorageStatus, StorageUtils} +import org.apache.spark.ui.{WebUIPage, UIUtils} import org.apache.spark.util.Utils /** Page showing storage details for a given RDD */ @@ -32,19 +32,28 @@ private[ui] class RDDPage(parent: StorageTab) extends WebUIPage("rdd") { def render(request: HttpServletRequest): Seq[Node] = { val parameterId = request.getParameter("id") require(parameterId != null && parameterId.nonEmpty, "Missing id parameter") + val rddId = parameterId.toInt - val rddStorageInfo = AllRDDResource.getRDDStorageInfo(rddId, listener,includeDetails = true) - .getOrElse { - // Rather than crashing, render an "RDD Not Found" page - return UIUtils.headerSparkPage("RDD Not Found", Seq[Node](), parent) - } + val storageStatusList = listener.storageStatusList + val rddInfo = listener.rddInfoList.find(_.id == rddId).getOrElse { + // Rather than crashing, render an "RDD Not Found" page + return UIUtils.headerSparkPage("RDD Not Found", Seq[Node](), parent) + } // Worker table - val workerTable = UIUtils.listingTable(workerHeader, workerRow, - rddStorageInfo.dataDistribution.get, id = Some("rdd-storage-by-worker-table")) + val workers = storageStatusList.map((rddId, _)) + val workerTable = UIUtils.listingTable(workerHeader, workerRow, workers, + id = Some("rdd-storage-by-worker-table")) // Block table - val blockTable = UIUtils.listingTable(blockHeader, blockRow, rddStorageInfo.partitions.get, + val blockLocations = StorageUtils.getRddBlockLocations(rddId, storageStatusList) + val blocks = storageStatusList + .flatMap(_.rddBlocksById(rddId)) + .sortWith(_._1.name < _._1.name) + .map { case (blockId, status) => + (blockId, status, blockLocations.get(blockId).getOrElse(Seq[String]("Unknown"))) + } + val blockTable = UIUtils.listingTable(blockHeader, blockRow, blocks, id = Some("rdd-storage-by-block-table")) val content = @@ -53,23 +62,23 @@ private[ui] class RDDPage(parent: StorageTab) extends WebUIPage("rdd") {
      • Storage Level: - {rddStorageInfo.storageLevel} + {rddInfo.storageLevel.description}
      • Cached Partitions: - {rddStorageInfo.numCachedPartitions} + {rddInfo.numCachedPartitions}
      • Total Partitions: - {rddStorageInfo.numPartitions} + {rddInfo.numPartitions}
      • Memory Size: - {Utils.bytesToString(rddStorageInfo.memoryUsed)} + {Utils.bytesToString(rddInfo.memSize)}
      • Disk Size: - {Utils.bytesToString(rddStorageInfo.diskUsed)} + {Utils.bytesToString(rddInfo.diskSize)}
      @@ -77,19 +86,19 @@ private[ui] class RDDPage(parent: StorageTab) extends WebUIPage("rdd") {
      -

      Data Distribution on {rddStorageInfo.dataDistribution.size} Executors

      +

      Data Distribution on {workers.size} Executors

      {workerTable}
      -

      {rddStorageInfo.partitions.size} Partitions

      +

      {blocks.size} Partitions

      {blockTable}
      ; - UIUtils.headerSparkPage("RDD Storage Info for " + rddStorageInfo.name, content, parent) + UIUtils.headerSparkPage("RDD Storage Info for " + rddInfo.name, content, parent) } /** Header fields for the worker table */ @@ -107,32 +116,34 @@ private[ui] class RDDPage(parent: StorageTab) extends WebUIPage("rdd") { "Executors") /** Render an HTML row representing a worker */ - private def workerRow(worker: RDDDataDistribution): Seq[Node] = { + private def workerRow(worker: (Int, StorageStatus)): Seq[Node] = { + val (rddId, status) = worker - {worker.address} + {status.blockManagerId.host + ":" + status.blockManagerId.port} - {Utils.bytesToString(worker.memoryUsed)} - ({Utils.bytesToString(worker.memoryRemaining)} Remaining) + {Utils.bytesToString(status.memUsedByRdd(rddId))} + ({Utils.bytesToString(status.memRemaining)} Remaining) - {Utils.bytesToString(worker.diskUsed)} + {Utils.bytesToString(status.diskUsedByRdd(rddId))} } /** Render an HTML row representing a block */ - private def blockRow(row: RDDPartitionInfo): Seq[Node] = { + private def blockRow(row: (BlockId, BlockStatus, Seq[String])): Seq[Node] = { + val (id, block, locations) = row - {row.blockName} + {id} - {row.storageLevel} + {block.storageLevel.description} - - {Utils.bytesToString(row.memoryUsed)} + + {Utils.bytesToString(block.memSize)} - - {Utils.bytesToString(row.diskUsed)} + + {Utils.bytesToString(block.diskSize)} - {row.executors.map(l => {l}
      )} + {locations.map(l => {l}
      )} } diff --git a/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala b/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala index 07db783c572cf..59dc6b547c1d8 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala @@ -22,7 +22,7 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node import org.apache.spark.storage.RDDInfo -import org.apache.spark.ui.{UIUtils, WebUIPage} +import org.apache.spark.ui.{WebUIPage, UIUtils} import org.apache.spark.util.Utils /** Page showing list of RDD's currently stored in the cluster */ diff --git a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala index 0351749700962..045bd784990d1 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala @@ -35,8 +35,6 @@ private[ui] class StorageTab(parent: SparkUI) extends SparkUITab(parent, "storag /** * :: DeveloperApi :: * A SparkListener that prepares information to be displayed on the BlockManagerUI. - * - * This class is thread-safe (unlike JobProgressListener) */ @DeveloperApi class StorageListener(storageStatusListener: StorageStatusListener) extends SparkListener { @@ -45,9 +43,7 @@ class StorageListener(storageStatusListener: StorageStatusListener) extends Spar def storageStatusList: Seq[StorageStatus] = storageStatusListener.storageStatusList /** Filter RDD info to include only those with cached partitions */ - def rddInfoList: Seq[RDDInfo] = synchronized { - _rddInfoMap.values.filter(_.numCachedPartitions > 0).toSeq - } + def rddInfoList: Seq[RDDInfo] = _rddInfoMap.values.filter(_.numCachedPartitions > 0).toSeq /** Update the storage info of the RDDs whose blocks are among the given updated blocks */ private def updateRDDInfo(updatedBlocks: Seq[(BlockId, BlockStatus)]): Unit = { diff --git a/core/src/test/resources/HistoryServerExpectations/applications/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/json_expectation deleted file mode 100644 index 6101177e4bb85..0000000000000 --- a/core/src/test/resources/HistoryServerExpectations/applications/json_expectation +++ /dev/null @@ -1,53 +0,0 @@ -[ { - "id" : "local-1427397477963", - "name" : "Spark shell", - "attempts" : [ { - "startTime" : "2015-03-26T19:17:57.184GMT", - "endTime" : "2015-03-26T19:20:02.949GMT", - "sparkUser" : "irashid", - "completed" : true - } ] -}, { - "id" : "local-1426533911241", - "name" : "Spark shell", - "attempts" : [ { - "attemptId" : "2", - "startTime" : "2015-03-17T23:11:50.242GMT", - "endTime" : "2015-03-17T23:12:25.177GMT", - "sparkUser" : "irashid", - "completed" : true - }, { - "attemptId" : "1", - "startTime" : "2015-03-16T19:25:10.242GMT", - "endTime" : "2015-03-16T19:25:45.177GMT", - "sparkUser" : "irashid", - "completed" : true - } ] -}, { - "id" : "local-1425081759269", - "name" : "Spark shell", - "attempts" : [ { - "startTime" : "2015-02-28T00:02:38.277GMT", - "endTime" : "2015-02-28T00:02:46.912GMT", - "sparkUser" : "irashid", - "completed" : true - } ] -}, { - "id" : "local-1422981780767", - "name" : "Spark shell", - "attempts" : [ { - "startTime" : "2015-02-03T16:42:59.720GMT", - "endTime" : "2015-02-03T16:43:08.731GMT", - "sparkUser" : "irashid", - "completed" : true - } ] -}, { - "id" : "local-1422981759269", - "name" : "Spark shell", - "attempts" : [ { - "startTime" : "2015-02-03T16:42:38.277GMT", - "endTime" : "2015-02-03T16:42:46.912GMT", - "sparkUser" : "irashid", - "completed" : true - } ] -} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/executors/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/executors/json_expectation deleted file mode 100644 index cb622e147249e..0000000000000 --- a/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/executors/json_expectation +++ /dev/null @@ -1,17 +0,0 @@ -[ { - "id" : "", - "hostPort" : "localhost:57971", - "rddBlocks" : 8, - "memoryUsed" : 28000128, - "diskUsed" : 0, - "activeTasks" : 0, - "failedTasks" : 1, - "completedTasks" : 31, - "totalTasks" : 32, - "totalDuration" : 8820, - "totalInputBytes" : 28000288, - "totalShuffleRead" : 0, - "totalShuffleWrite" : 13180, - "maxMemory" : 278302556, - "executorLogs" : { } -} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/jobs/0/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/jobs/0/json_expectation deleted file mode 100644 index 4a29072bdb6e4..0000000000000 --- a/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/jobs/0/json_expectation +++ /dev/null @@ -1,15 +0,0 @@ -{ - "jobId" : 0, - "name" : "count at :15", - "stageIds" : [ 0 ], - "status" : "SUCCEEDED", - "numTasks" : 8, - "numActiveTasks" : 0, - "numCompletedTasks" : 8, - "numSkippedTasks" : 8, - "numFailedTasks" : 0, - "numActiveStages" : 0, - "numCompletedStages" : 1, - "numSkippedStages" : 0, - "numFailedStages" : 0 -} \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/jobs/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/jobs/json_expectation deleted file mode 100644 index cab4750270dfa..0000000000000 --- a/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/jobs/json_expectation +++ /dev/null @@ -1,43 +0,0 @@ -[ { - "jobId" : 2, - "name" : "count at :17", - "stageIds" : [ 3 ], - "status" : "SUCCEEDED", - "numTasks" : 8, - "numActiveTasks" : 0, - "numCompletedTasks" : 8, - "numSkippedTasks" : 8, - "numFailedTasks" : 0, - "numActiveStages" : 0, - "numCompletedStages" : 1, - "numSkippedStages" : 0, - "numFailedStages" : 0 -}, { - "jobId" : 1, - "name" : "count at :20", - "stageIds" : [ 1, 2 ], - "status" : "FAILED", - "numTasks" : 16, - "numActiveTasks" : 0, - "numCompletedTasks" : 15, - "numSkippedTasks" : 15, - "numFailedTasks" : 1, - "numActiveStages" : 0, - "numCompletedStages" : 1, - "numSkippedStages" : 0, - "numFailedStages" : 1 -}, { - "jobId" : 0, - "name" : "count at :15", - "stageIds" : [ 0 ], - "status" : "SUCCEEDED", - "numTasks" : 8, - "numActiveTasks" : 0, - "numCompletedTasks" : 8, - "numSkippedTasks" : 8, - "numFailedTasks" : 0, - "numActiveStages" : 0, - "numCompletedStages" : 1, - "numSkippedStages" : 0, - "numFailedStages" : 0 -} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/jobs?status=succeeded&status=failed/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/jobs?status=succeeded&status=failed/json_expectation deleted file mode 100644 index cab4750270dfa..0000000000000 --- a/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/jobs?status=succeeded&status=failed/json_expectation +++ /dev/null @@ -1,43 +0,0 @@ -[ { - "jobId" : 2, - "name" : "count at :17", - "stageIds" : [ 3 ], - "status" : "SUCCEEDED", - "numTasks" : 8, - "numActiveTasks" : 0, - "numCompletedTasks" : 8, - "numSkippedTasks" : 8, - "numFailedTasks" : 0, - "numActiveStages" : 0, - "numCompletedStages" : 1, - "numSkippedStages" : 0, - "numFailedStages" : 0 -}, { - "jobId" : 1, - "name" : "count at :20", - "stageIds" : [ 1, 2 ], - "status" : "FAILED", - "numTasks" : 16, - "numActiveTasks" : 0, - "numCompletedTasks" : 15, - "numSkippedTasks" : 15, - "numFailedTasks" : 1, - "numActiveStages" : 0, - "numCompletedStages" : 1, - "numSkippedStages" : 0, - "numFailedStages" : 1 -}, { - "jobId" : 0, - "name" : "count at :15", - "stageIds" : [ 0 ], - "status" : "SUCCEEDED", - "numTasks" : 8, - "numActiveTasks" : 0, - "numCompletedTasks" : 8, - "numSkippedTasks" : 8, - "numFailedTasks" : 0, - "numActiveStages" : 0, - "numCompletedStages" : 1, - "numSkippedStages" : 0, - "numFailedStages" : 0 -} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/jobs?status=succeeded/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/jobs?status=succeeded/json_expectation deleted file mode 100644 index 6fd25befbf7e8..0000000000000 --- a/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/jobs?status=succeeded/json_expectation +++ /dev/null @@ -1,29 +0,0 @@ -[ { - "jobId" : 2, - "name" : "count at :17", - "stageIds" : [ 3 ], - "status" : "SUCCEEDED", - "numTasks" : 8, - "numActiveTasks" : 0, - "numCompletedTasks" : 8, - "numSkippedTasks" : 8, - "numFailedTasks" : 0, - "numActiveStages" : 0, - "numCompletedStages" : 1, - "numSkippedStages" : 0, - "numFailedStages" : 0 -}, { - "jobId" : 0, - "name" : "count at :15", - "stageIds" : [ 0 ], - "status" : "SUCCEEDED", - "numTasks" : 8, - "numActiveTasks" : 0, - "numCompletedTasks" : 8, - "numSkippedTasks" : 8, - "numFailedTasks" : 0, - "numActiveStages" : 0, - "numCompletedStages" : 1, - "numSkippedStages" : 0, - "numFailedStages" : 0 -} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/json_expectation deleted file mode 100644 index 07489ad96414a..0000000000000 --- a/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/json_expectation +++ /dev/null @@ -1,10 +0,0 @@ -{ - "id" : "local-1422981780767", - "name" : "Spark shell", - "attempts" : [ { - "startTime" : "2015-02-03T16:42:59.720GMT", - "endTime" : "2015-02-03T16:43:08.731GMT", - "sparkUser" : "irashid", - "completed" : true - } ] -} \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/stages/1/0/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/stages/1/0/json_expectation deleted file mode 100644 index 111cb8163eb3d..0000000000000 --- a/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/stages/1/0/json_expectation +++ /dev/null @@ -1,270 +0,0 @@ -{ - "status" : "COMPLETE", - "stageId" : 1, - "attemptId" : 0, - "numActiveTasks" : 0, - "numCompleteTasks" : 8, - "numFailedTasks" : 0, - "executorRunTime" : 3476, - "inputBytes" : 28000128, - "inputRecords" : 0, - "outputBytes" : 0, - "outputRecords" : 0, - "shuffleReadBytes" : 0, - "shuffleReadRecords" : 0, - "shuffleWriteBytes" : 13180, - "shuffleWriteRecords" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "name" : "map at :14", - "details" : "org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.(:14)\n$line10.$read$$iwC$$iwC$$iwC.(:19)\n$line10.$read$$iwC$$iwC.(:21)\n$line10.$read$$iwC.(:23)\n$line10.$read.(:25)\n$line10.$read$.(:29)\n$line10.$read$.()\n$line10.$eval$.(:7)\n$line10.$eval$.()\n$line10.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)", - "schedulingPool" : "default", - "accumulatorUpdates" : [ ], - "tasks" : { - "8" : { - "taskId" : 8, - "index" : 0, - "attempt" : 0, - "launchTime" : "2015-02-03T16:43:05.829GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 1, - "executorRunTime" : 435, - "resultSize" : 1902, - "jvmGcTime" : 19, - "resultSerializationTime" : 2, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "inputMetrics" : { - "bytesRead" : 3500016, - "recordsRead" : 0 - }, - "shuffleWriteMetrics" : { - "bytesWritten" : 1648, - "writeTime" : 94000, - "recordsWritten" : 0 - } - } - }, - "11" : { - "taskId" : 11, - "index" : 3, - "attempt" : 0, - "launchTime" : "2015-02-03T16:43:05.830GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 2, - "executorRunTime" : 434, - "resultSize" : 1902, - "jvmGcTime" : 19, - "resultSerializationTime" : 1, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "inputMetrics" : { - "bytesRead" : 3500016, - "recordsRead" : 0 - }, - "shuffleWriteMetrics" : { - "bytesWritten" : 1647, - "writeTime" : 83000, - "recordsWritten" : 0 - } - } - }, - "14" : { - "taskId" : 14, - "index" : 6, - "attempt" : 0, - "launchTime" : "2015-02-03T16:43:05.832GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 2, - "executorRunTime" : 434, - "resultSize" : 1902, - "jvmGcTime" : 19, - "resultSerializationTime" : 1, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "inputMetrics" : { - "bytesRead" : 3500016, - "recordsRead" : 0 - }, - "shuffleWriteMetrics" : { - "bytesWritten" : 1648, - "writeTime" : 88000, - "recordsWritten" : 0 - } - } - }, - "13" : { - "taskId" : 13, - "index" : 5, - "attempt" : 0, - "launchTime" : "2015-02-03T16:43:05.831GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 2, - "executorRunTime" : 434, - "resultSize" : 1902, - "jvmGcTime" : 19, - "resultSerializationTime" : 2, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "inputMetrics" : { - "bytesRead" : 3500016, - "recordsRead" : 0 - }, - "shuffleWriteMetrics" : { - "bytesWritten" : 1648, - "writeTime" : 73000, - "recordsWritten" : 0 - } - } - }, - "10" : { - "taskId" : 10, - "index" : 2, - "attempt" : 0, - "launchTime" : "2015-02-03T16:43:05.830GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 2, - "executorRunTime" : 434, - "resultSize" : 1902, - "jvmGcTime" : 19, - "resultSerializationTime" : 1, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "inputMetrics" : { - "bytesRead" : 3500016, - "recordsRead" : 0 - }, - "shuffleWriteMetrics" : { - "bytesWritten" : 1648, - "writeTime" : 76000, - "recordsWritten" : 0 - } - } - }, - "9" : { - "taskId" : 9, - "index" : 1, - "attempt" : 0, - "launchTime" : "2015-02-03T16:43:05.830GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 1, - "executorRunTime" : 436, - "resultSize" : 1902, - "jvmGcTime" : 19, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "inputMetrics" : { - "bytesRead" : 3500016, - "recordsRead" : 0 - }, - "shuffleWriteMetrics" : { - "bytesWritten" : 1648, - "writeTime" : 98000, - "recordsWritten" : 0 - } - } - }, - "12" : { - "taskId" : 12, - "index" : 4, - "attempt" : 0, - "launchTime" : "2015-02-03T16:43:05.831GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 2, - "executorRunTime" : 434, - "resultSize" : 1902, - "jvmGcTime" : 19, - "resultSerializationTime" : 1, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "inputMetrics" : { - "bytesRead" : 3500016, - "recordsRead" : 0 - }, - "shuffleWriteMetrics" : { - "bytesWritten" : 1645, - "writeTime" : 101000, - "recordsWritten" : 0 - } - } - }, - "15" : { - "taskId" : 15, - "index" : 7, - "attempt" : 0, - "launchTime" : "2015-02-03T16:43:05.833GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 1, - "executorRunTime" : 435, - "resultSize" : 1902, - "jvmGcTime" : 19, - "resultSerializationTime" : 1, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "inputMetrics" : { - "bytesRead" : 3500016, - "recordsRead" : 0 - }, - "shuffleWriteMetrics" : { - "bytesWritten" : 1648, - "writeTime" : 79000, - "recordsWritten" : 0 - } - } - } - }, - "executorSummary" : { - "" : { - "taskTime" : 3624, - "failedTasks" : 0, - "succeededTasks" : 8, - "inputBytes" : 28000128, - "outputBytes" : 0, - "shuffleRead" : 0, - "shuffleWrite" : 13180, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0 - } - } -} \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/stages/1/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/stages/1/json_expectation deleted file mode 100644 index ef339f89afa45..0000000000000 --- a/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/stages/1/json_expectation +++ /dev/null @@ -1,270 +0,0 @@ -[ { - "status" : "COMPLETE", - "stageId" : 1, - "attemptId" : 0, - "numActiveTasks" : 0, - "numCompleteTasks" : 8, - "numFailedTasks" : 0, - "executorRunTime" : 3476, - "inputBytes" : 28000128, - "inputRecords" : 0, - "outputBytes" : 0, - "outputRecords" : 0, - "shuffleReadBytes" : 0, - "shuffleReadRecords" : 0, - "shuffleWriteBytes" : 13180, - "shuffleWriteRecords" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "name" : "map at :14", - "details" : "org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.(:14)\n$line10.$read$$iwC$$iwC$$iwC.(:19)\n$line10.$read$$iwC$$iwC.(:21)\n$line10.$read$$iwC.(:23)\n$line10.$read.(:25)\n$line10.$read$.(:29)\n$line10.$read$.()\n$line10.$eval$.(:7)\n$line10.$eval$.()\n$line10.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)", - "schedulingPool" : "default", - "accumulatorUpdates" : [ ], - "tasks" : { - "8" : { - "taskId" : 8, - "index" : 0, - "attempt" : 0, - "launchTime" : "2015-02-03T16:43:05.829GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 1, - "executorRunTime" : 435, - "resultSize" : 1902, - "jvmGcTime" : 19, - "resultSerializationTime" : 2, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "inputMetrics" : { - "bytesRead" : 3500016, - "recordsRead" : 0 - }, - "shuffleWriteMetrics" : { - "bytesWritten" : 1648, - "writeTime" : 94000, - "recordsWritten" : 0 - } - } - }, - "11" : { - "taskId" : 11, - "index" : 3, - "attempt" : 0, - "launchTime" : "2015-02-03T16:43:05.830GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 2, - "executorRunTime" : 434, - "resultSize" : 1902, - "jvmGcTime" : 19, - "resultSerializationTime" : 1, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "inputMetrics" : { - "bytesRead" : 3500016, - "recordsRead" : 0 - }, - "shuffleWriteMetrics" : { - "bytesWritten" : 1647, - "writeTime" : 83000, - "recordsWritten" : 0 - } - } - }, - "14" : { - "taskId" : 14, - "index" : 6, - "attempt" : 0, - "launchTime" : "2015-02-03T16:43:05.832GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 2, - "executorRunTime" : 434, - "resultSize" : 1902, - "jvmGcTime" : 19, - "resultSerializationTime" : 1, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "inputMetrics" : { - "bytesRead" : 3500016, - "recordsRead" : 0 - }, - "shuffleWriteMetrics" : { - "bytesWritten" : 1648, - "writeTime" : 88000, - "recordsWritten" : 0 - } - } - }, - "13" : { - "taskId" : 13, - "index" : 5, - "attempt" : 0, - "launchTime" : "2015-02-03T16:43:05.831GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 2, - "executorRunTime" : 434, - "resultSize" : 1902, - "jvmGcTime" : 19, - "resultSerializationTime" : 2, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "inputMetrics" : { - "bytesRead" : 3500016, - "recordsRead" : 0 - }, - "shuffleWriteMetrics" : { - "bytesWritten" : 1648, - "writeTime" : 73000, - "recordsWritten" : 0 - } - } - }, - "10" : { - "taskId" : 10, - "index" : 2, - "attempt" : 0, - "launchTime" : "2015-02-03T16:43:05.830GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 2, - "executorRunTime" : 434, - "resultSize" : 1902, - "jvmGcTime" : 19, - "resultSerializationTime" : 1, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "inputMetrics" : { - "bytesRead" : 3500016, - "recordsRead" : 0 - }, - "shuffleWriteMetrics" : { - "bytesWritten" : 1648, - "writeTime" : 76000, - "recordsWritten" : 0 - } - } - }, - "9" : { - "taskId" : 9, - "index" : 1, - "attempt" : 0, - "launchTime" : "2015-02-03T16:43:05.830GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 1, - "executorRunTime" : 436, - "resultSize" : 1902, - "jvmGcTime" : 19, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "inputMetrics" : { - "bytesRead" : 3500016, - "recordsRead" : 0 - }, - "shuffleWriteMetrics" : { - "bytesWritten" : 1648, - "writeTime" : 98000, - "recordsWritten" : 0 - } - } - }, - "12" : { - "taskId" : 12, - "index" : 4, - "attempt" : 0, - "launchTime" : "2015-02-03T16:43:05.831GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 2, - "executorRunTime" : 434, - "resultSize" : 1902, - "jvmGcTime" : 19, - "resultSerializationTime" : 1, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "inputMetrics" : { - "bytesRead" : 3500016, - "recordsRead" : 0 - }, - "shuffleWriteMetrics" : { - "bytesWritten" : 1645, - "writeTime" : 101000, - "recordsWritten" : 0 - } - } - }, - "15" : { - "taskId" : 15, - "index" : 7, - "attempt" : 0, - "launchTime" : "2015-02-03T16:43:05.833GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 1, - "executorRunTime" : 435, - "resultSize" : 1902, - "jvmGcTime" : 19, - "resultSerializationTime" : 1, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "inputMetrics" : { - "bytesRead" : 3500016, - "recordsRead" : 0 - }, - "shuffleWriteMetrics" : { - "bytesWritten" : 1648, - "writeTime" : 79000, - "recordsWritten" : 0 - } - } - } - }, - "executorSummary" : { - "" : { - "taskTime" : 3624, - "failedTasks" : 0, - "succeededTasks" : 8, - "inputBytes" : 28000128, - "outputBytes" : 0, - "shuffleRead" : 0, - "shuffleWrite" : 13180, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0 - } - } -} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/stages/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/stages/json_expectation deleted file mode 100644 index 056fac7088594..0000000000000 --- a/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/stages/json_expectation +++ /dev/null @@ -1,89 +0,0 @@ -[ { - "status" : "COMPLETE", - "stageId" : 3, - "attemptId" : 0, - "numActiveTasks" : 0, - "numCompleteTasks" : 8, - "numFailedTasks" : 0, - "executorRunTime" : 162, - "inputBytes" : 160, - "inputRecords" : 0, - "outputBytes" : 0, - "outputRecords" : 0, - "shuffleReadBytes" : 0, - "shuffleReadRecords" : 0, - "shuffleWriteBytes" : 0, - "shuffleWriteRecords" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "name" : "count at :17", - "details" : "org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line19.$read$$iwC$$iwC$$iwC$$iwC.(:17)\n$line19.$read$$iwC$$iwC$$iwC.(:22)\n$line19.$read$$iwC$$iwC.(:24)\n$line19.$read$$iwC.(:26)\n$line19.$read.(:28)\n$line19.$read$.(:32)\n$line19.$read$.()\n$line19.$eval$.(:7)\n$line19.$eval$.()\n$line19.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)", - "schedulingPool" : "default", - "accumulatorUpdates" : [ ] -}, { - "status" : "COMPLETE", - "stageId" : 1, - "attemptId" : 0, - "numActiveTasks" : 0, - "numCompleteTasks" : 8, - "numFailedTasks" : 0, - "executorRunTime" : 3476, - "inputBytes" : 28000128, - "inputRecords" : 0, - "outputBytes" : 0, - "outputRecords" : 0, - "shuffleReadBytes" : 0, - "shuffleReadRecords" : 0, - "shuffleWriteBytes" : 13180, - "shuffleWriteRecords" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "name" : "map at :14", - "details" : "org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.(:14)\n$line10.$read$$iwC$$iwC$$iwC.(:19)\n$line10.$read$$iwC$$iwC.(:21)\n$line10.$read$$iwC.(:23)\n$line10.$read.(:25)\n$line10.$read$.(:29)\n$line10.$read$.()\n$line10.$eval$.(:7)\n$line10.$eval$.()\n$line10.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)", - "schedulingPool" : "default", - "accumulatorUpdates" : [ ] -}, { - "status" : "COMPLETE", - "stageId" : 0, - "attemptId" : 0, - "numActiveTasks" : 0, - "numCompleteTasks" : 8, - "numFailedTasks" : 0, - "executorRunTime" : 4338, - "inputBytes" : 0, - "inputRecords" : 0, - "outputBytes" : 0, - "outputRecords" : 0, - "shuffleReadBytes" : 0, - "shuffleReadRecords" : 0, - "shuffleWriteBytes" : 0, - "shuffleWriteRecords" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "name" : "count at :15", - "details" : "org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)", - "schedulingPool" : "default", - "accumulatorUpdates" : [ ] -}, { - "status" : "FAILED", - "stageId" : 2, - "attemptId" : 0, - "numActiveTasks" : 0, - "numCompleteTasks" : 7, - "numFailedTasks" : 1, - "executorRunTime" : 278, - "inputBytes" : 0, - "inputRecords" : 0, - "outputBytes" : 0, - "outputRecords" : 0, - "shuffleReadBytes" : 0, - "shuffleReadRecords" : 0, - "shuffleWriteBytes" : 0, - "shuffleWriteRecords" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "name" : "count at :20", - "details" : "org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line11.$read$$iwC$$iwC$$iwC$$iwC.(:20)\n$line11.$read$$iwC$$iwC$$iwC.(:25)\n$line11.$read$$iwC$$iwC.(:27)\n$line11.$read$$iwC.(:29)\n$line11.$read.(:31)\n$line11.$read$.(:35)\n$line11.$read$.()\n$line11.$eval$.(:7)\n$line11.$eval$.()\n$line11.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)", - "schedulingPool" : "default", - "accumulatorUpdates" : [ ] -} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/stages?status=complete/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/stages?status=complete/json_expectation deleted file mode 100644 index 31ac9beea8788..0000000000000 --- a/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/stages?status=complete/json_expectation +++ /dev/null @@ -1,67 +0,0 @@ -[ { - "status" : "COMPLETE", - "stageId" : 3, - "attemptId" : 0, - "numActiveTasks" : 0, - "numCompleteTasks" : 8, - "numFailedTasks" : 0, - "executorRunTime" : 162, - "inputBytes" : 160, - "inputRecords" : 0, - "outputBytes" : 0, - "outputRecords" : 0, - "shuffleReadBytes" : 0, - "shuffleReadRecords" : 0, - "shuffleWriteBytes" : 0, - "shuffleWriteRecords" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "name" : "count at :17", - "details" : "org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line19.$read$$iwC$$iwC$$iwC$$iwC.(:17)\n$line19.$read$$iwC$$iwC$$iwC.(:22)\n$line19.$read$$iwC$$iwC.(:24)\n$line19.$read$$iwC.(:26)\n$line19.$read.(:28)\n$line19.$read$.(:32)\n$line19.$read$.()\n$line19.$eval$.(:7)\n$line19.$eval$.()\n$line19.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)", - "schedulingPool" : "default", - "accumulatorUpdates" : [ ] -}, { - "status" : "COMPLETE", - "stageId" : 1, - "attemptId" : 0, - "numActiveTasks" : 0, - "numCompleteTasks" : 8, - "numFailedTasks" : 0, - "executorRunTime" : 3476, - "inputBytes" : 28000128, - "inputRecords" : 0, - "outputBytes" : 0, - "outputRecords" : 0, - "shuffleReadBytes" : 0, - "shuffleReadRecords" : 0, - "shuffleWriteBytes" : 13180, - "shuffleWriteRecords" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "name" : "map at :14", - "details" : "org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.(:14)\n$line10.$read$$iwC$$iwC$$iwC.(:19)\n$line10.$read$$iwC$$iwC.(:21)\n$line10.$read$$iwC.(:23)\n$line10.$read.(:25)\n$line10.$read$.(:29)\n$line10.$read$.()\n$line10.$eval$.(:7)\n$line10.$eval$.()\n$line10.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)", - "schedulingPool" : "default", - "accumulatorUpdates" : [ ] -}, { - "status" : "COMPLETE", - "stageId" : 0, - "attemptId" : 0, - "numActiveTasks" : 0, - "numCompleteTasks" : 8, - "numFailedTasks" : 0, - "executorRunTime" : 4338, - "inputBytes" : 0, - "inputRecords" : 0, - "outputBytes" : 0, - "outputRecords" : 0, - "shuffleReadBytes" : 0, - "shuffleReadRecords" : 0, - "shuffleWriteBytes" : 0, - "shuffleWriteRecords" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "name" : "count at :15", - "details" : "org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)", - "schedulingPool" : "default", - "accumulatorUpdates" : [ ] -} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/stages?status=failed/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/stages?status=failed/json_expectation deleted file mode 100644 index bff6a4f69d077..0000000000000 --- a/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/stages?status=failed/json_expectation +++ /dev/null @@ -1,23 +0,0 @@ -[ { - "status" : "FAILED", - "stageId" : 2, - "attemptId" : 0, - "numActiveTasks" : 0, - "numCompleteTasks" : 7, - "numFailedTasks" : 1, - "executorRunTime" : 278, - "inputBytes" : 0, - "inputRecords" : 0, - "outputBytes" : 0, - "outputRecords" : 0, - "shuffleReadBytes" : 0, - "shuffleReadRecords" : 0, - "shuffleWriteBytes" : 0, - "shuffleWriteRecords" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "name" : "count at :20", - "details" : "org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line11.$read$$iwC$$iwC$$iwC$$iwC.(:20)\n$line11.$read$$iwC$$iwC$$iwC.(:25)\n$line11.$read$$iwC$$iwC.(:27)\n$line11.$read$$iwC.(:29)\n$line11.$read.(:31)\n$line11.$read$.(:35)\n$line11.$read$.()\n$line11.$eval$.(:7)\n$line11.$eval$.()\n$line11.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)", - "schedulingPool" : "default", - "accumulatorUpdates" : [ ] -} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/storage/rdd/0/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/storage/rdd/0/json_expectation deleted file mode 100644 index 38b5328ffbb03..0000000000000 --- a/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/storage/rdd/0/json_expectation +++ /dev/null @@ -1,64 +0,0 @@ -{ - "id" : 0, - "name" : "0", - "numPartitions" : 8, - "numCachedPartitions" : 8, - "storageLevel" : "Memory Deserialized 1x Replicated", - "memoryUsed" : 28000128, - "diskUsed" : 0, - "dataDistribution" : [ { - "address" : "localhost:57971", - "memoryUsed" : 28000128, - "memoryRemaining" : 250302428, - "diskUsed" : 0 - } ], - "partitions" : [ { - "blockName" : "rdd_0_0", - "storageLevel" : "Memory Deserialized 1x Replicated", - "memoryUsed" : 3500016, - "diskUsed" : 0, - "executors" : [ "localhost:57971" ] - }, { - "blockName" : "rdd_0_1", - "storageLevel" : "Memory Deserialized 1x Replicated", - "memoryUsed" : 3500016, - "diskUsed" : 0, - "executors" : [ "localhost:57971" ] - }, { - "blockName" : "rdd_0_2", - "storageLevel" : "Memory Deserialized 1x Replicated", - "memoryUsed" : 3500016, - "diskUsed" : 0, - "executors" : [ "localhost:57971" ] - }, { - "blockName" : "rdd_0_3", - "storageLevel" : "Memory Deserialized 1x Replicated", - "memoryUsed" : 3500016, - "diskUsed" : 0, - "executors" : [ "localhost:57971" ] - }, { - "blockName" : "rdd_0_4", - "storageLevel" : "Memory Deserialized 1x Replicated", - "memoryUsed" : 3500016, - "diskUsed" : 0, - "executors" : [ "localhost:57971" ] - }, { - "blockName" : "rdd_0_5", - "storageLevel" : "Memory Deserialized 1x Replicated", - "memoryUsed" : 3500016, - "diskUsed" : 0, - "executors" : [ "localhost:57971" ] - }, { - "blockName" : "rdd_0_6", - "storageLevel" : "Memory Deserialized 1x Replicated", - "memoryUsed" : 3500016, - "diskUsed" : 0, - "executors" : [ "localhost:57971" ] - }, { - "blockName" : "rdd_0_7", - "storageLevel" : "Memory Deserialized 1x Replicated", - "memoryUsed" : 3500016, - "diskUsed" : 0, - "executors" : [ "localhost:57971" ] - } ] -} \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/storage/rdd/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/storage/rdd/json_expectation deleted file mode 100644 index f79a31022d214..0000000000000 --- a/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/storage/rdd/json_expectation +++ /dev/null @@ -1,9 +0,0 @@ -[ { - "id" : 0, - "name" : "0", - "numPartitions" : 8, - "numCachedPartitions" : 8, - "storageLevel" : "Memory Deserialized 1x Replicated", - "memoryUsed" : 28000128, - "diskUsed" : 0 -} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/1/jobs/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/1/jobs/json_expectation deleted file mode 100644 index 2e92e1fa0ec23..0000000000000 --- a/core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/1/jobs/json_expectation +++ /dev/null @@ -1,15 +0,0 @@ -[ { - "jobId" : 0, - "name" : "foreach at :15", - "stageIds" : [ 0 ], - "status" : "SUCCEEDED", - "numTasks" : 8, - "numActiveTasks" : 0, - "numCompletedTasks" : 8, - "numSkippedTasks" : 8, - "numFailedTasks" : 0, - "numActiveStages" : 0, - "numCompletedStages" : 1, - "numSkippedStages" : 0, - "numFailedStages" : 0 -} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/1/stages/0/0/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/1/stages/0/0/json_expectation deleted file mode 100644 index 32d5731676ad5..0000000000000 --- a/core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/1/stages/0/0/json_expectation +++ /dev/null @@ -1,242 +0,0 @@ -{ - "status" : "COMPLETE", - "stageId" : 0, - "attemptId" : 0, - "numActiveTasks" : 0, - "numCompleteTasks" : 8, - "numFailedTasks" : 0, - "executorRunTime" : 120, - "inputBytes" : 0, - "inputRecords" : 0, - "outputBytes" : 0, - "outputRecords" : 0, - "shuffleReadBytes" : 0, - "shuffleReadRecords" : 0, - "shuffleWriteBytes" : 0, - "shuffleWriteRecords" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "name" : "foreach at :15", - "details" : "org.apache.spark.rdd.RDD.foreach(RDD.scala:765)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)", - "schedulingPool" : "default", - "accumulatorUpdates" : [ { - "id" : 1, - "name" : "my counter", - "value" : "5050" - } ], - "tasks" : { - "2" : { - "taskId" : 2, - "index" : 2, - "attempt" : 0, - "launchTime" : "2015-03-16T19:25:36.522GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ { - "id" : 1, - "name" : "my counter", - "update" : "378", - "value" : "378" - } ], - "taskMetrics" : { - "executorDeserializeTime" : 13, - "executorRunTime" : 15, - "resultSize" : 697, - "jvmGcTime" : 0, - "resultSerializationTime" : 2, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0 - } - }, - "5" : { - "taskId" : 5, - "index" : 5, - "attempt" : 0, - "launchTime" : "2015-03-16T19:25:36.523GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ { - "id" : 1, - "name" : "my counter", - "update" : "897", - "value" : "3750" - } ], - "taskMetrics" : { - "executorDeserializeTime" : 12, - "executorRunTime" : 15, - "resultSize" : 697, - "jvmGcTime" : 0, - "resultSerializationTime" : 2, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0 - } - }, - "4" : { - "taskId" : 4, - "index" : 4, - "attempt" : 0, - "launchTime" : "2015-03-16T19:25:36.522GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ { - "id" : 1, - "name" : "my counter", - "update" : "678", - "value" : "2853" - } ], - "taskMetrics" : { - "executorDeserializeTime" : 12, - "executorRunTime" : 15, - "resultSize" : 697, - "jvmGcTime" : 0, - "resultSerializationTime" : 1, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0 - } - }, - "7" : { - "taskId" : 7, - "index" : 7, - "attempt" : 0, - "launchTime" : "2015-03-16T19:25:36.524GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ { - "id" : 1, - "name" : "my counter", - "update" : "1222", - "value" : "4972" - } ], - "taskMetrics" : { - "executorDeserializeTime" : 12, - "executorRunTime" : 15, - "resultSize" : 697, - "jvmGcTime" : 0, - "resultSerializationTime" : 2, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0 - } - }, - "1" : { - "taskId" : 1, - "index" : 1, - "attempt" : 0, - "launchTime" : "2015-03-16T19:25:36.521GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ { - "id" : 1, - "name" : "my counter", - "update" : "247", - "value" : "2175" - } ], - "taskMetrics" : { - "executorDeserializeTime" : 14, - "executorRunTime" : 15, - "resultSize" : 697, - "jvmGcTime" : 0, - "resultSerializationTime" : 2, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0 - } - }, - "3" : { - "taskId" : 3, - "index" : 3, - "attempt" : 0, - "launchTime" : "2015-03-16T19:25:36.522GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ { - "id" : 1, - "name" : "my counter", - "update" : "572", - "value" : "950" - } ], - "taskMetrics" : { - "executorDeserializeTime" : 13, - "executorRunTime" : 15, - "resultSize" : 697, - "jvmGcTime" : 0, - "resultSerializationTime" : 2, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0 - } - }, - "6" : { - "taskId" : 6, - "index" : 6, - "attempt" : 0, - "launchTime" : "2015-03-16T19:25:36.523GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ { - "id" : 1, - "name" : "my counter", - "update" : "978", - "value" : "1928" - } ], - "taskMetrics" : { - "executorDeserializeTime" : 12, - "executorRunTime" : 15, - "resultSize" : 697, - "jvmGcTime" : 0, - "resultSerializationTime" : 2, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0 - } - }, - "0" : { - "taskId" : 0, - "index" : 0, - "attempt" : 0, - "launchTime" : "2015-03-16T19:25:36.515GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ { - "id" : 1, - "name" : "my counter", - "update" : "78", - "value" : "5050" - } ], - "taskMetrics" : { - "executorDeserializeTime" : 14, - "executorRunTime" : 15, - "resultSize" : 697, - "jvmGcTime" : 0, - "resultSerializationTime" : 2, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0 - } - } - }, - "executorSummary" : { - "" : { - "taskTime" : 418, - "failedTasks" : 0, - "succeededTasks" : 8, - "inputBytes" : 0, - "outputBytes" : 0, - "shuffleRead" : 0, - "shuffleWrite" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0 - } - } -} \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/1/stages/0/0/taskList/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/1/stages/0/0/taskList/json_expectation deleted file mode 100644 index c3febc5fc9447..0000000000000 --- a/core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/1/stages/0/0/taskList/json_expectation +++ /dev/null @@ -1,193 +0,0 @@ -[ { - "taskId" : 0, - "index" : 0, - "attempt" : 0, - "launchTime" : "2015-03-16T19:25:36.515GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ { - "id" : 1, - "name" : "my counter", - "update" : "78", - "value" : "5050" - } ], - "taskMetrics" : { - "executorDeserializeTime" : 14, - "executorRunTime" : 15, - "resultSize" : 697, - "jvmGcTime" : 0, - "resultSerializationTime" : 2, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0 - } -}, { - "taskId" : 1, - "index" : 1, - "attempt" : 0, - "launchTime" : "2015-03-16T19:25:36.521GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ { - "id" : 1, - "name" : "my counter", - "update" : "247", - "value" : "2175" - } ], - "taskMetrics" : { - "executorDeserializeTime" : 14, - "executorRunTime" : 15, - "resultSize" : 697, - "jvmGcTime" : 0, - "resultSerializationTime" : 2, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0 - } -}, { - "taskId" : 2, - "index" : 2, - "attempt" : 0, - "launchTime" : "2015-03-16T19:25:36.522GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ { - "id" : 1, - "name" : "my counter", - "update" : "378", - "value" : "378" - } ], - "taskMetrics" : { - "executorDeserializeTime" : 13, - "executorRunTime" : 15, - "resultSize" : 697, - "jvmGcTime" : 0, - "resultSerializationTime" : 2, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0 - } -}, { - "taskId" : 3, - "index" : 3, - "attempt" : 0, - "launchTime" : "2015-03-16T19:25:36.522GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ { - "id" : 1, - "name" : "my counter", - "update" : "572", - "value" : "950" - } ], - "taskMetrics" : { - "executorDeserializeTime" : 13, - "executorRunTime" : 15, - "resultSize" : 697, - "jvmGcTime" : 0, - "resultSerializationTime" : 2, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0 - } -}, { - "taskId" : 4, - "index" : 4, - "attempt" : 0, - "launchTime" : "2015-03-16T19:25:36.522GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ { - "id" : 1, - "name" : "my counter", - "update" : "678", - "value" : "2853" - } ], - "taskMetrics" : { - "executorDeserializeTime" : 12, - "executorRunTime" : 15, - "resultSize" : 697, - "jvmGcTime" : 0, - "resultSerializationTime" : 1, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0 - } -}, { - "taskId" : 5, - "index" : 5, - "attempt" : 0, - "launchTime" : "2015-03-16T19:25:36.523GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ { - "id" : 1, - "name" : "my counter", - "update" : "897", - "value" : "3750" - } ], - "taskMetrics" : { - "executorDeserializeTime" : 12, - "executorRunTime" : 15, - "resultSize" : 697, - "jvmGcTime" : 0, - "resultSerializationTime" : 2, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0 - } -}, { - "taskId" : 6, - "index" : 6, - "attempt" : 0, - "launchTime" : "2015-03-16T19:25:36.523GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ { - "id" : 1, - "name" : "my counter", - "update" : "978", - "value" : "1928" - } ], - "taskMetrics" : { - "executorDeserializeTime" : 12, - "executorRunTime" : 15, - "resultSize" : 697, - "jvmGcTime" : 0, - "resultSerializationTime" : 2, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0 - } -}, { - "taskId" : 7, - "index" : 7, - "attempt" : 0, - "launchTime" : "2015-03-16T19:25:36.524GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ { - "id" : 1, - "name" : "my counter", - "update" : "1222", - "value" : "4972" - } ], - "taskMetrics" : { - "executorDeserializeTime" : 12, - "executorRunTime" : 15, - "resultSize" : 697, - "jvmGcTime" : 0, - "resultSerializationTime" : 2, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0 - } -} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/1/stages/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/1/stages/json_expectation deleted file mode 100644 index 79ccacd309693..0000000000000 --- a/core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/1/stages/json_expectation +++ /dev/null @@ -1,27 +0,0 @@ -[ { - "status" : "COMPLETE", - "stageId" : 0, - "attemptId" : 0, - "numActiveTasks" : 0, - "numCompleteTasks" : 8, - "numFailedTasks" : 0, - "executorRunTime" : 120, - "inputBytes" : 0, - "inputRecords" : 0, - "outputBytes" : 0, - "outputRecords" : 0, - "shuffleReadBytes" : 0, - "shuffleReadRecords" : 0, - "shuffleWriteBytes" : 0, - "shuffleWriteRecords" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "name" : "foreach at :15", - "details" : "org.apache.spark.rdd.RDD.foreach(RDD.scala:765)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)", - "schedulingPool" : "default", - "accumulatorUpdates" : [ { - "id" : 1, - "name" : "my counter", - "value" : "5050" - } ] -} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/2/jobs/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/2/jobs/json_expectation deleted file mode 100644 index 2e92e1fa0ec23..0000000000000 --- a/core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/2/jobs/json_expectation +++ /dev/null @@ -1,15 +0,0 @@ -[ { - "jobId" : 0, - "name" : "foreach at :15", - "stageIds" : [ 0 ], - "status" : "SUCCEEDED", - "numTasks" : 8, - "numActiveTasks" : 0, - "numCompletedTasks" : 8, - "numSkippedTasks" : 8, - "numFailedTasks" : 0, - "numActiveStages" : 0, - "numCompletedStages" : 1, - "numSkippedStages" : 0, - "numFailedStages" : 0 -} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/2/stages/0/0/taskList/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/2/stages/0/0/taskList/json_expectation deleted file mode 100644 index 56d667d88917c..0000000000000 --- a/core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/2/stages/0/0/taskList/json_expectation +++ /dev/null @@ -1,193 +0,0 @@ -[ { - "taskId" : 0, - "index" : 0, - "attempt" : 0, - "launchTime" : "2015-03-17T23:12:16.515GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ { - "id" : 1, - "name" : "my counter", - "update" : "78", - "value" : "5050" - } ], - "taskMetrics" : { - "executorDeserializeTime" : 14, - "executorRunTime" : 15, - "resultSize" : 697, - "jvmGcTime" : 0, - "resultSerializationTime" : 2, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0 - } -}, { - "taskId" : 1, - "index" : 1, - "attempt" : 0, - "launchTime" : "2015-03-17T23:12:16.521GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ { - "id" : 1, - "name" : "my counter", - "update" : "247", - "value" : "2175" - } ], - "taskMetrics" : { - "executorDeserializeTime" : 14, - "executorRunTime" : 15, - "resultSize" : 697, - "jvmGcTime" : 0, - "resultSerializationTime" : 2, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0 - } -}, { - "taskId" : 2, - "index" : 2, - "attempt" : 0, - "launchTime" : "2015-03-17T23:12:16.522GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ { - "id" : 1, - "name" : "my counter", - "update" : "378", - "value" : "378" - } ], - "taskMetrics" : { - "executorDeserializeTime" : 13, - "executorRunTime" : 15, - "resultSize" : 697, - "jvmGcTime" : 0, - "resultSerializationTime" : 2, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0 - } -}, { - "taskId" : 3, - "index" : 3, - "attempt" : 0, - "launchTime" : "2015-03-17T23:12:16.522GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ { - "id" : 1, - "name" : "my counter", - "update" : "572", - "value" : "950" - } ], - "taskMetrics" : { - "executorDeserializeTime" : 13, - "executorRunTime" : 15, - "resultSize" : 697, - "jvmGcTime" : 0, - "resultSerializationTime" : 2, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0 - } -}, { - "taskId" : 4, - "index" : 4, - "attempt" : 0, - "launchTime" : "2015-03-17T23:12:16.522GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ { - "id" : 1, - "name" : "my counter", - "update" : "678", - "value" : "2853" - } ], - "taskMetrics" : { - "executorDeserializeTime" : 12, - "executorRunTime" : 15, - "resultSize" : 697, - "jvmGcTime" : 0, - "resultSerializationTime" : 1, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0 - } -}, { - "taskId" : 5, - "index" : 5, - "attempt" : 0, - "launchTime" : "2015-03-17T23:12:16.523GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ { - "id" : 1, - "name" : "my counter", - "update" : "897", - "value" : "3750" - } ], - "taskMetrics" : { - "executorDeserializeTime" : 12, - "executorRunTime" : 15, - "resultSize" : 697, - "jvmGcTime" : 0, - "resultSerializationTime" : 2, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0 - } -}, { - "taskId" : 6, - "index" : 6, - "attempt" : 0, - "launchTime" : "2015-03-17T23:12:16.523GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ { - "id" : 1, - "name" : "my counter", - "update" : "978", - "value" : "1928" - } ], - "taskMetrics" : { - "executorDeserializeTime" : 12, - "executorRunTime" : 15, - "resultSize" : 697, - "jvmGcTime" : 0, - "resultSerializationTime" : 2, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0 - } -}, { - "taskId" : 7, - "index" : 7, - "attempt" : 0, - "launchTime" : "2015-03-17T23:12:16.524GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ { - "id" : 1, - "name" : "my counter", - "update" : "1222", - "value" : "4972" - } ], - "taskMetrics" : { - "executorDeserializeTime" : 12, - "executorRunTime" : 15, - "resultSize" : 697, - "jvmGcTime" : 0, - "resultSerializationTime" : 2, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0 - } -} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/json_expectation deleted file mode 100644 index 8f3d7160c723f..0000000000000 --- a/core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/json_expectation +++ /dev/null @@ -1,17 +0,0 @@ -{ - "id" : "local-1426533911241", - "name" : "Spark shell", - "attempts" : [ { - "attemptId" : "2", - "startTime" : "2015-03-17T23:11:50.242GMT", - "endTime" : "2015-03-17T23:12:25.177GMT", - "sparkUser" : "irashid", - "completed" : true - }, { - "attemptId" : "1", - "startTime" : "2015-03-16T19:25:10.242GMT", - "endTime" : "2015-03-16T19:25:45.177GMT", - "sparkUser" : "irashid", - "completed" : true - } ] -} \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskList/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskList/json_expectation deleted file mode 100644 index 140c76d7e5208..0000000000000 --- a/core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskList/json_expectation +++ /dev/null @@ -1,481 +0,0 @@ -[ { - "taskId" : 5010, - "index" : 0, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.571GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 1, - "executorRunTime" : 7, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9598, - "writeTime" : 318464, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5011, - "index" : 1, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.571GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 1, - "executorRunTime" : 6, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9601, - "writeTime" : 271674, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5012, - "index" : 2, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.571GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 7, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9601, - "writeTime" : 274899, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5013, - "index" : 3, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.571GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 11, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9597, - "writeTime" : 315224, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5014, - "index" : 4, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.571GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 12, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9599, - "writeTime" : 316561, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5015, - "index" : 5, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.571GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 7, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9598, - "writeTime" : 269443, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5016, - "index" : 6, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.571GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 5, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9600, - "writeTime" : 275977, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5017, - "index" : 7, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.571GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 6, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9600, - "writeTime" : 267412, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5018, - "index" : 8, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.579GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 6, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9600, - "writeTime" : 310279, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5019, - "index" : 9, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.579GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 6, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9601, - "writeTime" : 319028, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5020, - "index" : 10, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.579GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 1, - "executorRunTime" : 6, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9601, - "writeTime" : 306724, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5021, - "index" : 11, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.579GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 6, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9599, - "writeTime" : 280729, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5022, - "index" : 12, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.579GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 6, - "executorRunTime" : 5, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9601, - "writeTime" : 273837, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5023, - "index" : 13, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.579GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 6, - "executorRunTime" : 5, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9602, - "writeTime" : 282108, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5024, - "index" : 14, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.586GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 6, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9604, - "writeTime" : 266308, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5025, - "index" : 15, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.586GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 8, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9602, - "writeTime" : 330830, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5026, - "index" : 16, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.586GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 5, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9598, - "writeTime" : 247775, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5027, - "index" : 17, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.586GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 1, - "executorRunTime" : 5, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9601, - "writeTime" : 289503, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5028, - "index" : 18, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.592GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 6, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9601, - "writeTime" : 355545, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5029, - "index" : 19, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.593GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 7, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9598, - "writeTime" : 339081, - "recordsWritten" : 100 - } - } -} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskList?offset=10&length=50/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskList?offset=10&length=50/json_expectation deleted file mode 100644 index 0fa1ac28c9e90..0000000000000 --- a/core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskList?offset=10&length=50/json_expectation +++ /dev/null @@ -1,1201 +0,0 @@ -[ { - "taskId" : 5020, - "index" : 10, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.579GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 1, - "executorRunTime" : 6, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9601, - "writeTime" : 306724, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5021, - "index" : 11, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.579GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 6, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9599, - "writeTime" : 280729, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5022, - "index" : 12, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.579GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 6, - "executorRunTime" : 5, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9601, - "writeTime" : 273837, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5023, - "index" : 13, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.579GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 6, - "executorRunTime" : 5, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9602, - "writeTime" : 282108, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5024, - "index" : 14, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.586GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 6, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9604, - "writeTime" : 266308, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5025, - "index" : 15, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.586GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 8, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9602, - "writeTime" : 330830, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5026, - "index" : 16, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.586GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 5, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9598, - "writeTime" : 247775, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5027, - "index" : 17, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.586GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 1, - "executorRunTime" : 5, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9601, - "writeTime" : 289503, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5028, - "index" : 18, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.592GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 6, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9601, - "writeTime" : 355545, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5029, - "index" : 19, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.593GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 7, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9598, - "writeTime" : 339081, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5030, - "index" : 20, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.593GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 9, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9600, - "writeTime" : 367295, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5031, - "index" : 21, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.593GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 9, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9597, - "writeTime" : 493701, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5032, - "index" : 22, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.593GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 1, - "executorRunTime" : 6, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9601, - "writeTime" : 345941, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5033, - "index" : 23, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.593GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 6, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9600, - "writeTime" : 301382, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5034, - "index" : 24, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.594GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 6, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9603, - "writeTime" : 340547, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5035, - "index" : 25, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.600GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 1, - "executorRunTime" : 5, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9602, - "writeTime" : 324374, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5036, - "index" : 26, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.600GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 6, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9603, - "writeTime" : 321083, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5037, - "index" : 27, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.601GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 7, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9600, - "writeTime" : 318662, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5038, - "index" : 28, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.601GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 1, - "executorRunTime" : 6, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9599, - "writeTime" : 334376, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5039, - "index" : 29, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.601GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 7, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9602, - "writeTime" : 287042, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5040, - "index" : 30, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.601GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 6, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9601, - "writeTime" : 332938, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5041, - "index" : 31, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.602GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 4, - "executorRunTime" : 5, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9603, - "writeTime" : 255099, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5042, - "index" : 32, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.602GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 1, - "executorRunTime" : 6, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9598, - "writeTime" : 297534, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5043, - "index" : 33, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.607GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 6, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9601, - "writeTime" : 265929, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5044, - "index" : 34, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.607GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 5, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9601, - "writeTime" : 279851, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5045, - "index" : 35, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.608GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 5, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9600, - "writeTime" : 278965, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5046, - "index" : 36, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.609GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 1, - "executorRunTime" : 5, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9599, - "writeTime" : 340518, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5047, - "index" : 37, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.609GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 6, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9597, - "writeTime" : 402877, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5048, - "index" : 38, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.609GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 7, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9600, - "writeTime" : 265447, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5049, - "index" : 39, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.611GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 3, - "executorRunTime" : 23, - "resultSize" : 930, - "jvmGcTime" : 16, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9598, - "writeTime" : 304745, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5050, - "index" : 40, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.615GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 24, - "resultSize" : 930, - "jvmGcTime" : 16, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9600, - "writeTime" : 302371, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5051, - "index" : 41, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.615GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 1, - "executorRunTime" : 25, - "resultSize" : 930, - "jvmGcTime" : 16, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9600, - "writeTime" : 296177, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5052, - "index" : 42, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.615GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 24, - "resultSize" : 930, - "jvmGcTime" : 16, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9601, - "writeTime" : 962033, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5053, - "index" : 43, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.616GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 26, - "resultSize" : 930, - "jvmGcTime" : 16, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9600, - "writeTime" : 310335, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5054, - "index" : 44, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.616GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 110, - "resultSize" : 930, - "jvmGcTime" : 22, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9602, - "writeTime" : 299315, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5055, - "index" : 45, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.617GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 25, - "resultSize" : 930, - "jvmGcTime" : 16, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9603, - "writeTime" : 307821, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5056, - "index" : 46, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.617GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 17, - "executorRunTime" : 7, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9604, - "writeTime" : 306911, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5057, - "index" : 47, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.639GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 1, - "executorRunTime" : 6, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9600, - "writeTime" : 274191, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5058, - "index" : 48, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.640GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 48, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9598, - "writeTime" : 10442141, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5059, - "index" : 49, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.641GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 7, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9601, - "writeTime" : 300675, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5060, - "index" : 50, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.642GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 24, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9602, - "writeTime" : 812196, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5061, - "index" : 51, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.642GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 1, - "executorRunTime" : 6, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9600, - "writeTime" : 313090, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5062, - "index" : 52, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.642GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 17, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9598, - "writeTime" : 1572828, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5063, - "index" : 53, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.643GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 7, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9597, - "writeTime" : 312481, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5064, - "index" : 54, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.646GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 13, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9600, - "writeTime" : 296976, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5065, - "index" : 55, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.650GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 7, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9600, - "writeTime" : 670271, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5066, - "index" : 56, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.650GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 10, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9601, - "writeTime" : 1240309, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5067, - "index" : 57, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.650GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 9, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9601, - "writeTime" : 678632, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5068, - "index" : 58, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.659GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 7, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9600, - "writeTime" : 302888, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5069, - "index" : 59, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.660GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 7, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9610, - "writeTime" : 383153, - "recordsWritten" : 100 - } - } -} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskList?sortBy=-runtime/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskList?sortBy=-runtime/json_expectation deleted file mode 100644 index 63b131621c9c4..0000000000000 --- a/core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskList?sortBy=-runtime/json_expectation +++ /dev/null @@ -1,481 +0,0 @@ -[ { - "taskId" : 5186, - "index" : 176, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.880GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 147, - "resultSize" : 930, - "jvmGcTime" : 12, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9599, - "writeTime" : 317390, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5463, - "index" : 453, - "attempt" : 0, - "launchTime" : "2015-03-26T19:20:00.518GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 134, - "resultSize" : 930, - "jvmGcTime" : 5, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9598, - "writeTime" : 348853, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5461, - "index" : 451, - "attempt" : 0, - "launchTime" : "2015-03-26T19:20:00.516GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 134, - "resultSize" : 930, - "jvmGcTime" : 5, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9599, - "writeTime" : 413676, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5505, - "index" : 495, - "attempt" : 0, - "launchTime" : "2015-03-26T19:20:00.616GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 133, - "resultSize" : 930, - "jvmGcTime" : 11, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9602, - "writeTime" : 384212, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5672, - "index" : 662, - "attempt" : 0, - "launchTime" : "2015-03-26T19:20:01.016GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 1, - "executorRunTime" : 131, - "resultSize" : 930, - "jvmGcTime" : 4, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9602, - "writeTime" : 387955, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5865, - "index" : 855, - "attempt" : 0, - "launchTime" : "2015-03-26T19:20:01.514GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 131, - "resultSize" : 930, - "jvmGcTime" : 10, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9599, - "writeTime" : 1710924, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5415, - "index" : 405, - "attempt" : 0, - "launchTime" : "2015-03-26T19:20:00.389GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 130, - "resultSize" : 930, - "jvmGcTime" : 14, - "resultSerializationTime" : 1, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9597, - "writeTime" : 377101, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5462, - "index" : 452, - "attempt" : 0, - "launchTime" : "2015-03-26T19:20:00.518GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 128, - "resultSize" : 930, - "jvmGcTime" : 5, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9601, - "writeTime" : 306075, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5411, - "index" : 401, - "attempt" : 0, - "launchTime" : "2015-03-26T19:20:00.385GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 128, - "resultSize" : 930, - "jvmGcTime" : 14, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9602, - "writeTime" : 74684537, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5416, - "index" : 406, - "attempt" : 0, - "launchTime" : "2015-03-26T19:20:00.392GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 11, - "executorRunTime" : 123, - "resultSize" : 930, - "jvmGcTime" : 14, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9600, - "writeTime" : 365240, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5924, - "index" : 914, - "attempt" : 0, - "launchTime" : "2015-03-26T19:20:01.648GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 4, - "executorRunTime" : 121, - "resultSize" : 930, - "jvmGcTime" : 5, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9601, - "writeTime" : 335682, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5179, - "index" : 169, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.866GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 114, - "resultSize" : 930, - "jvmGcTime" : 5, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9601, - "writeTime" : 276876, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5243, - "index" : 233, - "attempt" : 0, - "launchTime" : "2015-03-26T19:20:00.018GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 113, - "resultSize" : 930, - "jvmGcTime" : 5, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9601, - "writeTime" : 12003053, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5054, - "index" : 44, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.616GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 110, - "resultSize" : 930, - "jvmGcTime" : 22, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9602, - "writeTime" : 299315, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5465, - "index" : 455, - "attempt" : 0, - "launchTime" : "2015-03-26T19:20:00.521GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 1, - "executorRunTime" : 108, - "resultSize" : 930, - "jvmGcTime" : 5, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9598, - "writeTime" : 318366, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5614, - "index" : 604, - "attempt" : 0, - "launchTime" : "2015-03-26T19:20:00.893GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 19, - "executorRunTime" : 108, - "resultSize" : 930, - "jvmGcTime" : 7, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9600, - "writeTime" : 309135, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5862, - "index" : 852, - "attempt" : 0, - "launchTime" : "2015-03-26T19:20:01.509GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 96, - "resultSize" : 930, - "jvmGcTime" : 5, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9598, - "writeTime" : 1052228, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5566, - "index" : 556, - "attempt" : 0, - "launchTime" : "2015-03-26T19:20:00.762GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 95, - "resultSize" : 930, - "jvmGcTime" : 6, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9600, - "writeTime" : 327089, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5567, - "index" : 557, - "attempt" : 0, - "launchTime" : "2015-03-26T19:20:00.768GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 94, - "resultSize" : 930, - "jvmGcTime" : 6, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9601, - "writeTime" : 374043, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5191, - "index" : 181, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.885GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 90, - "resultSize" : 930, - "jvmGcTime" : 5, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9598, - "writeTime" : 344199, - "recordsWritten" : 100 - } - } -} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskList?sortBy=DECREASING_RUNTIME/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskList?sortBy=DECREASING_RUNTIME/json_expectation deleted file mode 100644 index 63b131621c9c4..0000000000000 --- a/core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskList?sortBy=DECREASING_RUNTIME/json_expectation +++ /dev/null @@ -1,481 +0,0 @@ -[ { - "taskId" : 5186, - "index" : 176, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.880GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 147, - "resultSize" : 930, - "jvmGcTime" : 12, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9599, - "writeTime" : 317390, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5463, - "index" : 453, - "attempt" : 0, - "launchTime" : "2015-03-26T19:20:00.518GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 134, - "resultSize" : 930, - "jvmGcTime" : 5, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9598, - "writeTime" : 348853, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5461, - "index" : 451, - "attempt" : 0, - "launchTime" : "2015-03-26T19:20:00.516GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 134, - "resultSize" : 930, - "jvmGcTime" : 5, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9599, - "writeTime" : 413676, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5505, - "index" : 495, - "attempt" : 0, - "launchTime" : "2015-03-26T19:20:00.616GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 133, - "resultSize" : 930, - "jvmGcTime" : 11, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9602, - "writeTime" : 384212, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5672, - "index" : 662, - "attempt" : 0, - "launchTime" : "2015-03-26T19:20:01.016GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 1, - "executorRunTime" : 131, - "resultSize" : 930, - "jvmGcTime" : 4, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9602, - "writeTime" : 387955, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5865, - "index" : 855, - "attempt" : 0, - "launchTime" : "2015-03-26T19:20:01.514GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 131, - "resultSize" : 930, - "jvmGcTime" : 10, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9599, - "writeTime" : 1710924, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5415, - "index" : 405, - "attempt" : 0, - "launchTime" : "2015-03-26T19:20:00.389GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 130, - "resultSize" : 930, - "jvmGcTime" : 14, - "resultSerializationTime" : 1, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9597, - "writeTime" : 377101, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5462, - "index" : 452, - "attempt" : 0, - "launchTime" : "2015-03-26T19:20:00.518GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 128, - "resultSize" : 930, - "jvmGcTime" : 5, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9601, - "writeTime" : 306075, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5411, - "index" : 401, - "attempt" : 0, - "launchTime" : "2015-03-26T19:20:00.385GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 128, - "resultSize" : 930, - "jvmGcTime" : 14, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9602, - "writeTime" : 74684537, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5416, - "index" : 406, - "attempt" : 0, - "launchTime" : "2015-03-26T19:20:00.392GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 11, - "executorRunTime" : 123, - "resultSize" : 930, - "jvmGcTime" : 14, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9600, - "writeTime" : 365240, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5924, - "index" : 914, - "attempt" : 0, - "launchTime" : "2015-03-26T19:20:01.648GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 4, - "executorRunTime" : 121, - "resultSize" : 930, - "jvmGcTime" : 5, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9601, - "writeTime" : 335682, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5179, - "index" : 169, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.866GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 114, - "resultSize" : 930, - "jvmGcTime" : 5, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9601, - "writeTime" : 276876, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5243, - "index" : 233, - "attempt" : 0, - "launchTime" : "2015-03-26T19:20:00.018GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 113, - "resultSize" : 930, - "jvmGcTime" : 5, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9601, - "writeTime" : 12003053, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5054, - "index" : 44, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.616GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 110, - "resultSize" : 930, - "jvmGcTime" : 22, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9602, - "writeTime" : 299315, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5465, - "index" : 455, - "attempt" : 0, - "launchTime" : "2015-03-26T19:20:00.521GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 1, - "executorRunTime" : 108, - "resultSize" : 930, - "jvmGcTime" : 5, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9598, - "writeTime" : 318366, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5614, - "index" : 604, - "attempt" : 0, - "launchTime" : "2015-03-26T19:20:00.893GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 19, - "executorRunTime" : 108, - "resultSize" : 930, - "jvmGcTime" : 7, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9600, - "writeTime" : 309135, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5862, - "index" : 852, - "attempt" : 0, - "launchTime" : "2015-03-26T19:20:01.509GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 96, - "resultSize" : 930, - "jvmGcTime" : 5, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9598, - "writeTime" : 1052228, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5566, - "index" : 556, - "attempt" : 0, - "launchTime" : "2015-03-26T19:20:00.762GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 95, - "resultSize" : 930, - "jvmGcTime" : 6, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9600, - "writeTime" : 327089, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5567, - "index" : 557, - "attempt" : 0, - "launchTime" : "2015-03-26T19:20:00.768GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 94, - "resultSize" : 930, - "jvmGcTime" : 6, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9601, - "writeTime" : 374043, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5191, - "index" : 181, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.885GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 90, - "resultSize" : 930, - "jvmGcTime" : 5, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9598, - "writeTime" : 344199, - "recordsWritten" : 100 - } - } -} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskList?sortBy=runtime/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskList?sortBy=runtime/json_expectation deleted file mode 100644 index 329065b835dd0..0000000000000 --- a/core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskList?sortBy=runtime/json_expectation +++ /dev/null @@ -1,481 +0,0 @@ -[ { - "taskId" : 5610, - "index" : 600, - "attempt" : 0, - "launchTime" : "2015-03-26T19:20:00.884GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 5, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9601, - "writeTime" : 285253, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5511, - "index" : 501, - "attempt" : 0, - "launchTime" : "2015-03-26T19:20:00.634GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 1, - "executorRunTime" : 5, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9599, - "writeTime" : 277358, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5027, - "index" : 17, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.586GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 1, - "executorRunTime" : 5, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9601, - "writeTime" : 289503, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5828, - "index" : 818, - "attempt" : 0, - "launchTime" : "2015-03-26T19:20:01.438GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 3, - "executorRunTime" : 5, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9601, - "writeTime" : 291997, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5173, - "index" : 163, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.861GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 5, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9599, - "writeTime" : 267617, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5185, - "index" : 175, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.880GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 5, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9600, - "writeTime" : 285928, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5553, - "index" : 543, - "attempt" : 0, - "launchTime" : "2015-03-26T19:20:00.719GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 1, - "executorRunTime" : 5, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9601, - "writeTime" : 298472, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5077, - "index" : 67, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.670GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 1, - "executorRunTime" : 5, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9614, - "writeTime" : 259332, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5831, - "index" : 821, - "attempt" : 0, - "launchTime" : "2015-03-26T19:20:01.442GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 1, - "executorRunTime" : 5, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9597, - "writeTime" : 281603, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5194, - "index" : 184, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.903GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 1, - "executorRunTime" : 5, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9600, - "writeTime" : 286315, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5481, - "index" : 471, - "attempt" : 0, - "launchTime" : "2015-03-26T19:20:00.561GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 1, - "executorRunTime" : 5, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9613, - "writeTime" : 328893, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5035, - "index" : 25, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.600GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 1, - "executorRunTime" : 5, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9602, - "writeTime" : 324374, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5134, - "index" : 124, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.782GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 5, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9599, - "writeTime" : 279741, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5044, - "index" : 34, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.607GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 5, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9601, - "writeTime" : 279851, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5026, - "index" : 16, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.586GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 5, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9598, - "writeTime" : 247775, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5331, - "index" : 321, - "attempt" : 0, - "launchTime" : "2015-03-26T19:20:00.208GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 1, - "executorRunTime" : 5, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9601, - "writeTime" : 313699, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5343, - "index" : 333, - "attempt" : 0, - "launchTime" : "2015-03-26T19:20:00.241GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 0, - "executorRunTime" : 5, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9601, - "writeTime" : 260259, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5606, - "index" : 596, - "attempt" : 0, - "launchTime" : "2015-03-26T19:20:00.878GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 1, - "executorRunTime" : 5, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9599, - "writeTime" : 310521, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5041, - "index" : 31, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.602GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 4, - "executorRunTime" : 5, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9603, - "writeTime" : 255099, - "recordsWritten" : 100 - } - } -}, { - "taskId" : 5023, - "index" : 13, - "attempt" : 0, - "launchTime" : "2015-03-26T19:19:59.579GMT", - "executorId" : "", - "host" : "localhost", - "taskLocality" : "PROCESS_LOCAL", - "speculative" : false, - "accumulatorUpdates" : [ ], - "taskMetrics" : { - "executorDeserializeTime" : 6, - "executorRunTime" : 5, - "resultSize" : 930, - "jvmGcTime" : 0, - "resultSerializationTime" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, - "shuffleWriteMetrics" : { - "bytesWritten" : 9602, - "writeTime" : 282108, - "recordsWritten" : 100 - } - } -} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskSummary/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskSummary/json_expectation deleted file mode 100644 index 46f32c19abed5..0000000000000 --- a/core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskSummary/json_expectation +++ /dev/null @@ -1,15 +0,0 @@ -{ - "quantiles" : [ 0.05, 0.25, 0.5, 0.75, 0.95 ], - "executorDeserializeTime" : [ 0.0, 0.0, 0.0, 1.0, 3.0 ], - "executorRunTime" : [ 6.0, 6.0, 7.0, 10.0, 48.0 ], - "resultSize" : [ 930.0, 930.0, 930.0, 930.0, 930.0 ], - "jvmGcTime" : [ 0.0, 0.0, 0.0, 0.0, 7.0 ], - "resultSerializationTime" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], - "memoryBytesSpilled" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], - "diskBytesSpilled" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], - "shuffleWriteMetrics" : { - "writeBytes" : [ 9598.0, 9599.0, 9600.0, 9601.0, 9603.0 ], - "writeRecords" : [ 100.0, 100.0, 100.0, 100.0, 100.0 ], - "writeTime" : [ 278805.0, 308809.0, 327953.0, 367487.0, 944783.0 ] - } -} \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskSummary?quantiles=0.01,0.5,0.99/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskSummary?quantiles=0.01,0.5,0.99/json_expectation deleted file mode 100644 index 65b46388ced87..0000000000000 --- a/core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskSummary?quantiles=0.01,0.5,0.99/json_expectation +++ /dev/null @@ -1,15 +0,0 @@ -{ - "quantiles" : [ 0.01, 0.5, 0.99 ], - "executorDeserializeTime" : [ 0.0, 0.0, 25.0 ], - "executorRunTime" : [ 5.0, 7.0, 123.0 ], - "resultSize" : [ 930.0, 930.0, 930.0 ], - "jvmGcTime" : [ 0.0, 0.0, 14.0 ], - "resultSerializationTime" : [ 0.0, 0.0, 1.0 ], - "memoryBytesSpilled" : [ 0.0, 0.0, 0.0 ], - "diskBytesSpilled" : [ 0.0, 0.0, 0.0 ], - "shuffleWriteMetrics" : { - "writeBytes" : [ 9597.0, 9600.0, 9614.0 ], - "writeRecords" : [ 100.0, 100.0, 100.0 ], - "writeTime" : [ 260991.0, 327953.0, 8980526.0 ] - } -} \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/applications?maxDate=2015-02-03T10:42:40.000CST/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications?maxDate=2015-02-03T10:42:40.000CST/json_expectation deleted file mode 100644 index 483632a3956ed..0000000000000 --- a/core/src/test/resources/HistoryServerExpectations/applications?maxDate=2015-02-03T10:42:40.000CST/json_expectation +++ /dev/null @@ -1,10 +0,0 @@ -[ { - "id" : "local-1422981759269", - "name" : "Spark shell", - "attempts" : [ { - "startTime" : "2015-02-03T16:42:38.277GMT", - "endTime" : "2015-02-03T16:42:46.912GMT", - "sparkUser" : "irashid", - "completed" : true - } ] -} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/applications?maxDate=2015-02-10/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications?maxDate=2015-02-10/json_expectation deleted file mode 100644 index 4b85690fd9199..0000000000000 --- a/core/src/test/resources/HistoryServerExpectations/applications?maxDate=2015-02-10/json_expectation +++ /dev/null @@ -1,19 +0,0 @@ -[ { - "id" : "local-1422981780767", - "name" : "Spark shell", - "attempts" : [ { - "startTime" : "2015-02-03T16:42:59.720GMT", - "endTime" : "2015-02-03T16:43:08.731GMT", - "sparkUser" : "irashid", - "completed" : true - } ] -}, { - "id" : "local-1422981759269", - "name" : "Spark shell", - "attempts" : [ { - "startTime" : "2015-02-03T16:42:38.277GMT", - "endTime" : "2015-02-03T16:42:46.912GMT", - "sparkUser" : "irashid", - "completed" : true - } ] -} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/applications?minDate=2015-02-10/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications?minDate=2015-02-10/json_expectation deleted file mode 100644 index e67f25252e934..0000000000000 --- a/core/src/test/resources/HistoryServerExpectations/applications?minDate=2015-02-10/json_expectation +++ /dev/null @@ -1,35 +0,0 @@ -[ { - "id" : "local-1427397477963", - "name" : "Spark shell", - "attempts" : [ { - "startTime" : "2015-03-26T19:17:57.184GMT", - "endTime" : "2015-03-26T19:20:02.949GMT", - "sparkUser" : "irashid", - "completed" : true - } ] -}, { - "id" : "local-1426533911241", - "name" : "Spark shell", - "attempts" : [ { - "attemptId" : "2", - "startTime" : "2015-03-17T23:11:50.242GMT", - "endTime" : "2015-03-17T23:12:25.177GMT", - "sparkUser" : "irashid", - "completed" : true - }, { - "attemptId" : "1", - "startTime" : "2015-03-16T19:25:10.242GMT", - "endTime" : "2015-03-16T19:25:45.177GMT", - "sparkUser" : "irashid", - "completed" : true - } ] -}, { - "id" : "local-1425081759269", - "name" : "Spark shell", - "attempts" : [ { - "startTime" : "2015-02-28T00:02:38.277GMT", - "endTime" : "2015-02-28T00:02:46.912GMT", - "sparkUser" : "irashid", - "completed" : true - } ] -} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/applications?status=completed/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications?status=completed/json_expectation deleted file mode 100644 index 6101177e4bb85..0000000000000 --- a/core/src/test/resources/HistoryServerExpectations/applications?status=completed/json_expectation +++ /dev/null @@ -1,53 +0,0 @@ -[ { - "id" : "local-1427397477963", - "name" : "Spark shell", - "attempts" : [ { - "startTime" : "2015-03-26T19:17:57.184GMT", - "endTime" : "2015-03-26T19:20:02.949GMT", - "sparkUser" : "irashid", - "completed" : true - } ] -}, { - "id" : "local-1426533911241", - "name" : "Spark shell", - "attempts" : [ { - "attemptId" : "2", - "startTime" : "2015-03-17T23:11:50.242GMT", - "endTime" : "2015-03-17T23:12:25.177GMT", - "sparkUser" : "irashid", - "completed" : true - }, { - "attemptId" : "1", - "startTime" : "2015-03-16T19:25:10.242GMT", - "endTime" : "2015-03-16T19:25:45.177GMT", - "sparkUser" : "irashid", - "completed" : true - } ] -}, { - "id" : "local-1425081759269", - "name" : "Spark shell", - "attempts" : [ { - "startTime" : "2015-02-28T00:02:38.277GMT", - "endTime" : "2015-02-28T00:02:46.912GMT", - "sparkUser" : "irashid", - "completed" : true - } ] -}, { - "id" : "local-1422981780767", - "name" : "Spark shell", - "attempts" : [ { - "startTime" : "2015-02-03T16:42:59.720GMT", - "endTime" : "2015-02-03T16:43:08.731GMT", - "sparkUser" : "irashid", - "completed" : true - } ] -}, { - "id" : "local-1422981759269", - "name" : "Spark shell", - "attempts" : [ { - "startTime" : "2015-02-03T16:42:38.277GMT", - "endTime" : "2015-02-03T16:42:46.912GMT", - "sparkUser" : "irashid", - "completed" : true - } ] -} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/applications?status=running/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications?status=running/json_expectation deleted file mode 100644 index 8878e547a7984..0000000000000 --- a/core/src/test/resources/HistoryServerExpectations/applications?status=running/json_expectation +++ /dev/null @@ -1 +0,0 @@ -[ ] \ No newline at end of file diff --git a/core/src/test/resources/spark-events/local-1422981759269/APPLICATION_COMPLETE b/core/src/test/resources/spark-events/local-1422981759269/APPLICATION_COMPLETE deleted file mode 100755 index e69de29bb2d1d..0000000000000 diff --git a/core/src/test/resources/spark-events/local-1422981759269/EVENT_LOG_1 b/core/src/test/resources/spark-events/local-1422981759269/EVENT_LOG_1 deleted file mode 100755 index 4794e56d1107a..0000000000000 --- a/core/src/test/resources/spark-events/local-1422981759269/EVENT_LOG_1 +++ /dev/null @@ -1,88 +0,0 @@ -{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"","Host":"localhost","Port":57967},"Maximum Memory":278302556,"Timestamp":1422981759407} -{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre","Java Version":"1.7.0_67 (Oracle Corporation)","Scala Version":"version 2.10.4"},"Spark Properties":{"spark.driver.host":"192.168.1.103","spark.eventLog.enabled":"true","spark.driver.port":"57965","spark.repl.class.uri":"http://192.168.1.103:57964","spark.jars":"","spark.app.name":"Spark shell","spark.scheduler.mode":"FIFO","spark.executor.id":"driver","spark.master":"local[*]","spark.fileserver.uri":"http://192.168.1.103:57966","spark.tachyonStore.folderName":"spark-fd6c823a-8a18-4113-8306-1fa7bb623a7f","spark.app.id":"local-1422981759269"},"System Properties":{"java.io.tmpdir":"/var/folders/36/m29jw1z95qv4ywb1c4n0rz000000gp/T/","line.separator":"\n","path.separator":":","sun.management.compiler":"HotSpot 64-Bit Tiered Compilers","SPARK_SUBMIT":"true","sun.cpu.endian":"little","java.specification.version":"1.7","java.vm.specification.name":"Java Virtual Machine Specification","java.vendor":"Oracle Corporation","java.vm.specification.version":"1.7","user.home":"/Users/irashid","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","ftp.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","sun.arch.data.model":"64","sun.boot.library.path":"/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib","user.dir":"/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4","java.library.path":"/Users/irashid/Library/Java/Extensions:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java:.","sun.cpu.isalist":"","os.arch":"x86_64","java.vm.version":"24.65-b04","java.endorsed.dirs":"/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/endorsed","java.runtime.version":"1.7.0_67-b01","java.vm.info":"mixed mode","java.ext.dirs":"/Users/irashid/Library/Java/Extensions:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/ext:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java","java.runtime.name":"Java(TM) SE Runtime Environment","file.separator":"/","java.class.version":"51.0","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/resources.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/rt.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/sunrsasign.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/jsse.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/jce.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/charsets.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/jfr.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/classes","file.encoding":"UTF-8","user.timezone":"America/Chicago","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"10.9.5","sun.os.patch.level":"unknown","gopherProxySet":"false","java.vm.specification.vendor":"Oracle Corporation","user.country":"US","sun.jnu.encoding":"UTF-8","http.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","user.language":"en","socksNonProxyHosts":"local|*.local|169.254/16|*.169.254/16","java.vendor.url":"http://java.oracle.com/","java.awt.printerjob":"sun.lwawt.macosx.CPrinterJob","java.awt.graphicsenv":"sun.awt.CGraphicsEnvironment","awt.toolkit":"sun.lwawt.macosx.LWCToolkit","os.name":"Mac OS X","java.vm.vendor":"Oracle Corporation","java.vendor.url.bug":"http://bugreport.sun.com/bugreport/","user.name":"irashid","java.vm.name":"Java HotSpot(TM) 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --class org.apache.spark.repl.Main --conf spark.eventLog.enabled=true spark-shell","java.home":"/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre","java.version":"1.7.0_67","sun.io.unicode.encoding":"UnicodeBig"},"Classpath Entries":{"/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/spark-assembly-1.2.0-hadoop2.4.0.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-api-jdo-3.2.6.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-rdbms-3.2.9.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-core-3.2.10.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/conf":"System Classpath"}} -{"Event":"SparkListenerApplicationStart","App Name":"Spark shell","App ID":"local-1422981759269","Timestamp":1422981758277,"User":"irashid"} -{"Event":"SparkListenerJobStart","Job ID":0,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"count at :15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}],"Stage IDs":[0]} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"count at :15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1422981762075,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1422981762081,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1422981762081,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1422981762082,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1422981762083,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1422981762084,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1422981762084,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1422981762085,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1422981762084,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981762632,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":521,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_6","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1422981762081,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981762633,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":520,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_2","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1422981762082,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981762634,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":521,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_3","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1422981762084,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981762634,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":522,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_5","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1422981762083,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981762635,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":522,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_4","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1422981762075,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981762636,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":11,"Executor Run Time":522,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1422981762085,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981762636,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":9,"Executor Run Time":521,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_7","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1422981762081,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981762637,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":11,"Executor Run Time":522,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_1","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"count at :15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":8,"Memory Size":28000128,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1422981762069,"Completion Time":1422981762637,"Accumulables":[]}} -{"Event":"SparkListenerJobEnd","Job ID":0,"Job Result":{"Result":"JobSucceeded"}} -{"Event":"SparkListenerJobStart","Job ID":1,"Stage Infos":[{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"map at :14","Number of Tasks":8,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.(:14)\n$line10.$read$$iwC$$iwC$$iwC.(:19)\n$line10.$read$$iwC$$iwC.(:21)\n$line10.$read$$iwC.(:23)\n$line10.$read.(:25)\n$line10.$read$.(:29)\n$line10.$read$.()\n$line10.$eval$.(:7)\n$line10.$eval$.()\n$line10.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]},{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"first at :17","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.first(RDD.scala:1093)\n$line11.$read$$iwC$$iwC$$iwC$$iwC.(:17)\n$line11.$read$$iwC$$iwC$$iwC.(:22)\n$line11.$read$$iwC$$iwC.(:24)\n$line11.$read$$iwC.(:26)\n$line11.$read.(:28)\n$line11.$read$.(:32)\n$line11.$read$.()\n$line11.$eval$.(:7)\n$line11.$eval$.()\n$line11.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}],"Stage IDs":[1,2]} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"map at :14","Number of Tasks":8,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.(:14)\n$line10.$read$$iwC$$iwC$$iwC.(:19)\n$line10.$read$$iwC$$iwC.(:21)\n$line10.$read$$iwC.(:23)\n$line10.$read.(:25)\n$line10.$read$.(:29)\n$line10.$read$.()\n$line10.$eval$.(:7)\n$line10.$eval$.()\n$line10.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":8,"Index":0,"Attempt":0,"Launch Time":1422981763578,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":9,"Index":1,"Attempt":0,"Launch Time":1422981763578,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":10,"Index":2,"Attempt":0,"Launch Time":1422981763579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":11,"Index":3,"Attempt":0,"Launch Time":1422981763579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":12,"Index":4,"Attempt":0,"Launch Time":1422981763580,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":13,"Index":5,"Attempt":0,"Launch Time":1422981763580,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":14,"Index":6,"Attempt":0,"Launch Time":1422981763581,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":15,"Index":7,"Attempt":0,"Launch Time":1422981763581,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":10,"Index":2,"Attempt":0,"Launch Time":1422981763579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764001,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":406,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":138000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} -{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":9,"Index":1,"Attempt":0,"Launch Time":1422981763578,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764002,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":407,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":106000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} -{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":12,"Index":4,"Attempt":0,"Launch Time":1422981763580,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764002,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":407,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1645,"Shuffle Write Time":99000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} -{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":8,"Index":0,"Attempt":0,"Launch Time":1422981763578,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764003,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":407,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":123000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} -{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":15,"Index":7,"Attempt":0,"Launch Time":1422981763581,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764003,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":406,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":108000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} -{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":11,"Index":3,"Attempt":0,"Launch Time":1422981763579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764004,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":407,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1647,"Shuffle Write Time":97000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} -{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":14,"Index":6,"Attempt":0,"Launch Time":1422981763581,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764004,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":407,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":132000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} -{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":13,"Index":5,"Attempt":0,"Launch Time":1422981763580,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764005,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":407,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":81000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"map at :14","Number of Tasks":8,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.(:14)\n$line10.$read$$iwC$$iwC$$iwC.(:19)\n$line10.$read$$iwC$$iwC.(:21)\n$line10.$read$$iwC.(:23)\n$line10.$read.(:25)\n$line10.$read$.(:29)\n$line10.$read$.()\n$line10.$eval$.(:7)\n$line10.$eval$.()\n$line10.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1422981763578,"Completion Time":1422981764005,"Accumulables":[]}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"first at :17","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.first(RDD.scala:1093)\n$line11.$read$$iwC$$iwC$$iwC$$iwC.(:17)\n$line11.$read$$iwC$$iwC$$iwC.(:22)\n$line11.$read$$iwC$$iwC.(:24)\n$line11.$read$$iwC.(:26)\n$line11.$read.(:28)\n$line11.$read$.(:32)\n$line11.$read$.()\n$line11.$eval$.(:7)\n$line11.$eval$.()\n$line11.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":16,"Index":0,"Attempt":0,"Launch Time":1422981764014,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":16,"Index":0,"Attempt":0,"Launch Time":1422981764014,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764045,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":28,"Result Size":1013,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"first at :17","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.first(RDD.scala:1093)\n$line11.$read$$iwC$$iwC$$iwC$$iwC.(:17)\n$line11.$read$$iwC$$iwC$$iwC.(:22)\n$line11.$read$$iwC$$iwC.(:24)\n$line11.$read$$iwC.(:26)\n$line11.$read.(:28)\n$line11.$read$.(:32)\n$line11.$read$.()\n$line11.$eval$.(:7)\n$line11.$eval$.()\n$line11.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1422981764014,"Completion Time":1422981764045,"Accumulables":[]}} -{"Event":"SparkListenerJobEnd","Job ID":1,"Job Result":{"Result":"JobSucceeded"}} -{"Event":"SparkListenerJobStart","Job ID":2,"Stage Infos":[{"Stage ID":3,"Stage Attempt ID":0,"Stage Name":"map at :14","Number of Tasks":8,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.(:14)\n$line10.$read$$iwC$$iwC$$iwC.(:19)\n$line10.$read$$iwC$$iwC.(:21)\n$line10.$read$$iwC.(:23)\n$line10.$read.(:25)\n$line10.$read$.(:29)\n$line10.$read$.()\n$line10.$eval$.(:7)\n$line10.$eval$.()\n$line10.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]},{"Stage ID":4,"Stage Attempt ID":0,"Stage Name":"saveAsTextFile at :19","Number of Tasks":8,"RDD Info":[{"RDD ID":3,"Name":"3","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.saveAsTextFile(RDD.scala:1164)\n$line13.$read$$iwC$$iwC$$iwC$$iwC.(:19)\n$line13.$read$$iwC$$iwC$$iwC.(:24)\n$line13.$read$$iwC$$iwC.(:26)\n$line13.$read$$iwC.(:28)\n$line13.$read.(:30)\n$line13.$read$.(:34)\n$line13.$read$.()\n$line13.$eval$.(:7)\n$line13.$eval$.()\n$line13.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}],"Stage IDs":[3,4]} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":4,"Stage Attempt ID":0,"Stage Name":"saveAsTextFile at :19","Number of Tasks":8,"RDD Info":[{"RDD ID":3,"Name":"3","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.saveAsTextFile(RDD.scala:1164)\n$line13.$read$$iwC$$iwC$$iwC$$iwC.(:19)\n$line13.$read$$iwC$$iwC$$iwC.(:24)\n$line13.$read$$iwC$$iwC.(:26)\n$line13.$read$$iwC.(:28)\n$line13.$read.(:30)\n$line13.$read$.(:34)\n$line13.$read$.()\n$line13.$eval$.(:7)\n$line13.$eval$.()\n$line13.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":17,"Index":0,"Attempt":0,"Launch Time":1422981764396,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":18,"Index":1,"Attempt":0,"Launch Time":1422981764396,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":19,"Index":2,"Attempt":0,"Launch Time":1422981764397,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":20,"Index":3,"Attempt":0,"Launch Time":1422981764397,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":21,"Index":4,"Attempt":0,"Launch Time":1422981764398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":22,"Index":5,"Attempt":0,"Launch Time":1422981764398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":23,"Index":6,"Attempt":0,"Launch Time":1422981764398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":24,"Index":7,"Attempt":0,"Launch Time":1422981764399,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":24,"Index":7,"Attempt":0,"Launch Time":1422981764399,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764642,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":240,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":19,"Index":2,"Attempt":0,"Launch Time":1422981764397,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764643,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":241,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":21,"Index":4,"Attempt":0,"Launch Time":1422981764398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764643,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":240,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":22,"Index":5,"Attempt":0,"Launch Time":1422981764398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764645,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":241,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":23,"Index":6,"Attempt":0,"Launch Time":1422981764398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764645,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":241,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":17,"Index":0,"Attempt":0,"Launch Time":1422981764396,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764646,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":243,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":18,"Index":1,"Attempt":0,"Launch Time":1422981764396,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764646,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":243,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":20,"Index":3,"Attempt":0,"Launch Time":1422981764397,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764648,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":247,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":4,"Stage Attempt ID":0,"Stage Name":"saveAsTextFile at :19","Number of Tasks":8,"RDD Info":[{"RDD ID":3,"Name":"3","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.saveAsTextFile(RDD.scala:1164)\n$line13.$read$$iwC$$iwC$$iwC$$iwC.(:19)\n$line13.$read$$iwC$$iwC$$iwC.(:24)\n$line13.$read$$iwC$$iwC.(:26)\n$line13.$read$$iwC.(:28)\n$line13.$read.(:30)\n$line13.$read$.(:34)\n$line13.$read$.()\n$line13.$eval$.(:7)\n$line13.$eval$.()\n$line13.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1422981764396,"Completion Time":1422981764648,"Accumulables":[]}} -{"Event":"SparkListenerJobEnd","Job ID":2,"Job Result":{"Result":"JobSucceeded"}} -{"Event":"SparkListenerJobStart","Job ID":3,"Stage Infos":[{"Stage ID":5,"Stage Attempt ID":0,"Stage Name":"count at :17","Number of Tasks":8,"RDD Info":[{"RDD ID":5,"Name":"/Users/irashid/spark-examples/tmp_data/sums","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":4,"Name":"/Users/irashid/spark-examples/tmp_data/sums","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.(:17)\n$line15.$read$$iwC$$iwC$$iwC.(:22)\n$line15.$read$$iwC$$iwC.(:24)\n$line15.$read$$iwC.(:26)\n$line15.$read.(:28)\n$line15.$read$.(:32)\n$line15.$read$.()\n$line15.$eval$.(:7)\n$line15.$eval$.()\n$line15.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}],"Stage IDs":[5]} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":5,"Stage Attempt ID":0,"Stage Name":"count at :17","Number of Tasks":8,"RDD Info":[{"RDD ID":5,"Name":"/Users/irashid/spark-examples/tmp_data/sums","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":4,"Name":"/Users/irashid/spark-examples/tmp_data/sums","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.(:17)\n$line15.$read$$iwC$$iwC$$iwC.(:22)\n$line15.$read$$iwC$$iwC.(:24)\n$line15.$read$$iwC.(:26)\n$line15.$read.(:28)\n$line15.$read$.(:32)\n$line15.$read$.()\n$line15.$eval$.(:7)\n$line15.$eval$.()\n$line15.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":25,"Index":0,"Attempt":0,"Launch Time":1422981765026,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":26,"Index":1,"Attempt":0,"Launch Time":1422981765026,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":27,"Index":2,"Attempt":0,"Launch Time":1422981765027,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":28,"Index":3,"Attempt":0,"Launch Time":1422981765027,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":29,"Index":4,"Attempt":0,"Launch Time":1422981765027,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":30,"Index":5,"Attempt":0,"Launch Time":1422981765028,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":31,"Index":6,"Attempt":0,"Launch Time":1422981765028,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":32,"Index":7,"Attempt":0,"Launch Time":1422981765029,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":27,"Index":2,"Attempt":0,"Launch Time":1422981765027,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981765045,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} -{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":30,"Index":5,"Attempt":0,"Launch Time":1422981765028,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981765046,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} -{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":25,"Index":0,"Attempt":0,"Launch Time":1422981765026,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981765046,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":12,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":32}}} -{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":29,"Index":4,"Attempt":0,"Launch Time":1422981765027,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981765047,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} -{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":28,"Index":3,"Attempt":0,"Launch Time":1422981765027,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981765047,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":12,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} -{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":32,"Index":7,"Attempt":0,"Launch Time":1422981765029,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981765048,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} -{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":31,"Index":6,"Attempt":0,"Launch Time":1422981765028,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981765048,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":14,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} -{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":26,"Index":1,"Attempt":0,"Launch Time":1422981765026,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981765049,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":12,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":32}}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":5,"Stage Attempt ID":0,"Stage Name":"count at :17","Number of Tasks":8,"RDD Info":[{"RDD ID":5,"Name":"/Users/irashid/spark-examples/tmp_data/sums","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":4,"Name":"/Users/irashid/spark-examples/tmp_data/sums","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.(:17)\n$line15.$read$$iwC$$iwC$$iwC.(:22)\n$line15.$read$$iwC$$iwC.(:24)\n$line15.$read$$iwC.(:26)\n$line15.$read.(:28)\n$line15.$read$.(:32)\n$line15.$read$.()\n$line15.$eval$.(:7)\n$line15.$eval$.()\n$line15.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1422981765026,"Completion Time":1422981765050,"Accumulables":[]}} -{"Event":"SparkListenerJobEnd","Job ID":3,"Job Result":{"Result":"JobSucceeded"}} -{"Event":"SparkListenerApplicationEnd","Timestamp":1422981766912} diff --git a/core/src/test/resources/spark-events/local-1422981759269/SPARK_VERSION_1.2.0 b/core/src/test/resources/spark-events/local-1422981759269/SPARK_VERSION_1.2.0 deleted file mode 100755 index e69de29bb2d1d..0000000000000 diff --git a/core/src/test/resources/spark-events/local-1422981780767/APPLICATION_COMPLETE b/core/src/test/resources/spark-events/local-1422981780767/APPLICATION_COMPLETE deleted file mode 100755 index e69de29bb2d1d..0000000000000 diff --git a/core/src/test/resources/spark-events/local-1422981780767/EVENT_LOG_1 b/core/src/test/resources/spark-events/local-1422981780767/EVENT_LOG_1 deleted file mode 100755 index f14a000bf2c28..0000000000000 --- a/core/src/test/resources/spark-events/local-1422981780767/EVENT_LOG_1 +++ /dev/null @@ -1,82 +0,0 @@ -{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"","Host":"localhost","Port":57971},"Maximum Memory":278302556,"Timestamp":1422981780906} -{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre","Java Version":"1.7.0_67 (Oracle Corporation)","Scala Version":"version 2.10.4"},"Spark Properties":{"spark.driver.host":"192.168.1.103","spark.eventLog.enabled":"true","spark.driver.port":"57969","spark.repl.class.uri":"http://192.168.1.103:57968","spark.jars":"","spark.app.name":"Spark shell","spark.scheduler.mode":"FIFO","spark.executor.id":"driver","spark.master":"local[*]","spark.fileserver.uri":"http://192.168.1.103:57970","spark.tachyonStore.folderName":"spark-3f19daee-844c-41d0-a3fc-5e3e508f9731","spark.app.id":"local-1422981780767"},"System Properties":{"java.io.tmpdir":"/var/folders/36/m29jw1z95qv4ywb1c4n0rz000000gp/T/","line.separator":"\n","path.separator":":","sun.management.compiler":"HotSpot 64-Bit Tiered Compilers","SPARK_SUBMIT":"true","sun.cpu.endian":"little","java.specification.version":"1.7","java.vm.specification.name":"Java Virtual Machine Specification","java.vendor":"Oracle Corporation","java.vm.specification.version":"1.7","user.home":"/Users/irashid","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","ftp.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","sun.arch.data.model":"64","sun.boot.library.path":"/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib","user.dir":"/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4","java.library.path":"/Users/irashid/Library/Java/Extensions:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java:.","sun.cpu.isalist":"","os.arch":"x86_64","java.vm.version":"24.65-b04","java.endorsed.dirs":"/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/endorsed","java.runtime.version":"1.7.0_67-b01","java.vm.info":"mixed mode","java.ext.dirs":"/Users/irashid/Library/Java/Extensions:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/ext:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java","java.runtime.name":"Java(TM) SE Runtime Environment","file.separator":"/","java.class.version":"51.0","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/resources.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/rt.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/sunrsasign.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/jsse.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/jce.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/charsets.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/jfr.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/classes","file.encoding":"UTF-8","user.timezone":"America/Chicago","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"10.9.5","sun.os.patch.level":"unknown","gopherProxySet":"false","java.vm.specification.vendor":"Oracle Corporation","user.country":"US","sun.jnu.encoding":"UTF-8","http.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","user.language":"en","socksNonProxyHosts":"local|*.local|169.254/16|*.169.254/16","java.vendor.url":"http://java.oracle.com/","java.awt.printerjob":"sun.lwawt.macosx.CPrinterJob","java.awt.graphicsenv":"sun.awt.CGraphicsEnvironment","awt.toolkit":"sun.lwawt.macosx.LWCToolkit","os.name":"Mac OS X","java.vm.vendor":"Oracle Corporation","java.vendor.url.bug":"http://bugreport.sun.com/bugreport/","user.name":"irashid","java.vm.name":"Java HotSpot(TM) 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --class org.apache.spark.repl.Main --conf spark.eventLog.enabled=true spark-shell","java.home":"/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre","java.version":"1.7.0_67","sun.io.unicode.encoding":"UnicodeBig"},"Classpath Entries":{"/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/spark-assembly-1.2.0-hadoop2.4.0.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-api-jdo-3.2.6.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-rdbms-3.2.9.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-core-3.2.10.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/conf":"System Classpath"}} -{"Event":"SparkListenerApplicationStart","App Name":"Spark shell","App ID":"local-1422981780767","Timestamp":1422981779720,"User":"irashid"} -{"Event":"SparkListenerJobStart","Job ID":0,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"count at :15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}],"Stage IDs":[0]} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"count at :15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1422981784234,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1422981784240,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1422981784240,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1422981784241,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1422981784241,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1422981784242,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1422981784242,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1422981784243,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1422981784241,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981784812,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":11,"Executor Run Time":543,"Result Size":1268,"JVM GC Time":25,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_3","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1422981784240,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981784814,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":542,"Result Size":1268,"JVM GC Time":25,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_1","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1422981784234,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981784816,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":542,"Result Size":1268,"JVM GC Time":25,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1422981784243,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981784816,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":11,"Executor Run Time":543,"Result Size":1268,"JVM GC Time":25,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_7","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1422981784242,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981784817,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":541,"Result Size":1268,"JVM GC Time":25,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_5","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1422981784241,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981784817,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":11,"Executor Run Time":542,"Result Size":1268,"JVM GC Time":25,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_4","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1422981784242,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981784818,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":11,"Executor Run Time":543,"Result Size":1268,"JVM GC Time":25,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_6","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1422981784240,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981784818,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":11,"Executor Run Time":542,"Result Size":1268,"JVM GC Time":25,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_2","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"count at :15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":8,"Memory Size":28000128,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1422981784228,"Completion Time":1422981784819,"Accumulables":[]}} -{"Event":"SparkListenerJobEnd","Job ID":0,"Job Result":{"Result":"JobSucceeded"}} -{"Event":"SparkListenerJobStart","Job ID":1,"Stage Infos":[{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"map at :14","Number of Tasks":8,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.(:14)\n$line10.$read$$iwC$$iwC$$iwC.(:19)\n$line10.$read$$iwC$$iwC.(:21)\n$line10.$read$$iwC.(:23)\n$line10.$read.(:25)\n$line10.$read$.(:29)\n$line10.$read$.()\n$line10.$eval$.(:7)\n$line10.$eval$.()\n$line10.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]},{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"count at :20","Number of Tasks":8,"RDD Info":[{"RDD ID":3,"Name":"3","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line11.$read$$iwC$$iwC$$iwC$$iwC.(:20)\n$line11.$read$$iwC$$iwC$$iwC.(:25)\n$line11.$read$$iwC$$iwC.(:27)\n$line11.$read$$iwC.(:29)\n$line11.$read.(:31)\n$line11.$read$.(:35)\n$line11.$read$.()\n$line11.$eval$.(:7)\n$line11.$eval$.()\n$line11.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}],"Stage IDs":[1,2]} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"map at :14","Number of Tasks":8,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.(:14)\n$line10.$read$$iwC$$iwC$$iwC.(:19)\n$line10.$read$$iwC$$iwC.(:21)\n$line10.$read$$iwC.(:23)\n$line10.$read.(:25)\n$line10.$read$.(:29)\n$line10.$read$.()\n$line10.$eval$.(:7)\n$line10.$eval$.()\n$line10.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":8,"Index":0,"Attempt":0,"Launch Time":1422981785829,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":9,"Index":1,"Attempt":0,"Launch Time":1422981785830,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":10,"Index":2,"Attempt":0,"Launch Time":1422981785830,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":11,"Index":3,"Attempt":0,"Launch Time":1422981785830,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":12,"Index":4,"Attempt":0,"Launch Time":1422981785831,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":13,"Index":5,"Attempt":0,"Launch Time":1422981785831,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":14,"Index":6,"Attempt":0,"Launch Time":1422981785832,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":15,"Index":7,"Attempt":0,"Launch Time":1422981785833,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":14,"Index":6,"Attempt":0,"Launch Time":1422981785832,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786282,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":434,"Result Size":1902,"JVM GC Time":19,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":88000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} -{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":8,"Index":0,"Attempt":0,"Launch Time":1422981785829,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786283,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":435,"Result Size":1902,"JVM GC Time":19,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":94000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} -{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":15,"Index":7,"Attempt":0,"Launch Time":1422981785833,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786283,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":435,"Result Size":1902,"JVM GC Time":19,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":79000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} -{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":13,"Index":5,"Attempt":0,"Launch Time":1422981785831,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786283,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":434,"Result Size":1902,"JVM GC Time":19,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":73000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} -{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":11,"Index":3,"Attempt":0,"Launch Time":1422981785830,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786284,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":434,"Result Size":1902,"JVM GC Time":19,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1647,"Shuffle Write Time":83000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} -{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":9,"Index":1,"Attempt":0,"Launch Time":1422981785830,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786284,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":436,"Result Size":1902,"JVM GC Time":19,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":98000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} -{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":12,"Index":4,"Attempt":0,"Launch Time":1422981785831,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786285,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":434,"Result Size":1902,"JVM GC Time":19,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1645,"Shuffle Write Time":101000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} -{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":10,"Index":2,"Attempt":0,"Launch Time":1422981785830,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786286,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":434,"Result Size":1902,"JVM GC Time":19,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":76000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"map at :14","Number of Tasks":8,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.(:14)\n$line10.$read$$iwC$$iwC$$iwC.(:19)\n$line10.$read$$iwC$$iwC.(:21)\n$line10.$read$$iwC.(:23)\n$line10.$read.(:25)\n$line10.$read$.(:29)\n$line10.$read$.()\n$line10.$eval$.(:7)\n$line10.$eval$.()\n$line10.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1422981785829,"Completion Time":1422981786286,"Accumulables":[]}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"count at :20","Number of Tasks":8,"RDD Info":[{"RDD ID":3,"Name":"3","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line11.$read$$iwC$$iwC$$iwC$$iwC.(:20)\n$line11.$read$$iwC$$iwC$$iwC.(:25)\n$line11.$read$$iwC$$iwC.(:27)\n$line11.$read$$iwC.(:29)\n$line11.$read.(:31)\n$line11.$read$.(:35)\n$line11.$read$.()\n$line11.$eval$.(:7)\n$line11.$eval$.()\n$line11.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":16,"Index":0,"Attempt":0,"Launch Time":1422981786296,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":17,"Index":1,"Attempt":0,"Launch Time":1422981786297,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":18,"Index":2,"Attempt":0,"Launch Time":1422981786297,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":19,"Index":3,"Attempt":0,"Launch Time":1422981786297,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":20,"Index":4,"Attempt":0,"Launch Time":1422981786298,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":21,"Index":5,"Attempt":0,"Launch Time":1422981786298,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":22,"Index":6,"Attempt":0,"Launch Time":1422981786298,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":23,"Index":7,"Attempt":0,"Launch Time":1422981786299,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":20,"Index":4,"Attempt":0,"Launch Time":1422981786298,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786337,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":34,"Result Size":862,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} -{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":21,"Index":5,"Attempt":0,"Launch Time":1422981786298,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786339,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":35,"Result Size":862,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} -{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":17,"Index":1,"Attempt":0,"Launch Time":1422981786297,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786340,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":35,"Result Size":862,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":1,"Remote Bytes Read":0}}} -{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":18,"Index":2,"Attempt":0,"Launch Time":1422981786297,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786340,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":34,"Result Size":862,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} -{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":16,"Index":0,"Attempt":0,"Launch Time":1422981786296,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786340,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":35,"Result Size":862,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} -{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":23,"Index":7,"Attempt":0,"Launch Time":1422981786299,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786341,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":35,"Result Size":862,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} -{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":22,"Index":6,"Attempt":0,"Launch Time":1422981786298,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786342,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":34,"Result Size":862,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} -{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"ExceptionFailure","Class Name":"java.lang.RuntimeException","Description":"got a 3, failing","Stack Trace":[{"Declaring Class":"$line11.$read$$iwC$$iwC$$iwC$$iwC$$anonfun$1","Method Name":"apply","File Name":"","Line Number":18},{"Declaring Class":"$line11.$read$$iwC$$iwC$$iwC$$iwC$$anonfun$1","Method Name":"apply","File Name":"","Line Number":17},{"Declaring Class":"scala.collection.Iterator$$anon$11","Method Name":"next","File Name":"Iterator.scala","Line Number":328},{"Declaring Class":"org.apache.spark.util.Utils$","Method Name":"getIteratorSize","File Name":"Utils.scala","Line Number":1311},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":910},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":910},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$4","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1314},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$4","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1314},{"Declaring Class":"org.apache.spark.scheduler.ResultTask","Method Name":"runTask","File Name":"ResultTask.scala","Line Number":61},{"Declaring Class":"org.apache.spark.scheduler.Task","Method Name":"run","File Name":"Task.scala","Line Number":56},{"Declaring Class":"org.apache.spark.executor.Executor$TaskRunner","Method Name":"run","File Name":"Executor.scala","Line Number":196},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor","Method Name":"runWorker","File Name":"ThreadPoolExecutor.java","Line Number":1145},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor$Worker","Method Name":"run","File Name":"ThreadPoolExecutor.java","Line Number":615},{"Declaring Class":"java.lang.Thread","Method Name":"run","File Name":"Thread.java","Line Number":745}],"Full Stack Trace":"java.lang.RuntimeException: got a 3, failing\n\tat $line11.$read$$iwC$$iwC$$iwC$$iwC$$anonfun$1.apply(:18)\n\tat $line11.$read$$iwC$$iwC$$iwC$$iwC$$anonfun$1.apply(:17)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:328)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1311)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:910)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:910)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$4.apply(SparkContext.scala:1314)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$4.apply(SparkContext.scala:1314)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:61)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:56)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:196)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)\n\tat java.lang.Thread.run(Thread.java:745)\n","Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":36,"Result Size":0,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}},"Task Info":{"Task ID":19,"Index":3,"Attempt":0,"Launch Time":1422981786297,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786343,"Failed":true,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":36,"Result Size":0,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"count at :20","Number of Tasks":8,"RDD Info":[{"RDD ID":3,"Name":"3","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line11.$read$$iwC$$iwC$$iwC$$iwC.(:20)\n$line11.$read$$iwC$$iwC$$iwC.(:25)\n$line11.$read$$iwC$$iwC.(:27)\n$line11.$read$$iwC.(:29)\n$line11.$read.(:31)\n$line11.$read$.(:35)\n$line11.$read$.()\n$line11.$eval$.(:7)\n$line11.$eval$.()\n$line11.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1422981786296,"Completion Time":1422981786347,"Failure Reason":"Job aborted due to stage failure: Task 3 in stage 2.0 failed 1 times, most recent failure: Lost task 3.0 in stage 2.0 (TID 19, localhost): java.lang.RuntimeException: got a 3, failing\n\tat $line11.$read$$iwC$$iwC$$iwC$$iwC$$anonfun$1.apply(:18)\n\tat $line11.$read$$iwC$$iwC$$iwC$$iwC$$anonfun$1.apply(:17)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:328)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1311)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:910)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:910)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$4.apply(SparkContext.scala:1314)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$4.apply(SparkContext.scala:1314)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:61)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:56)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:196)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)\n\tat java.lang.Thread.run(Thread.java:745)\n\nDriver stacktrace:","Accumulables":[]}} -{"Event":"SparkListenerJobEnd","Job ID":1,"Job Result":{"Result":"JobFailed","Exception":{"Message":"Job aborted due to stage failure: Task 3 in stage 2.0 failed 1 times, most recent failure: Lost task 3.0 in stage 2.0 (TID 19, localhost): java.lang.RuntimeException: got a 3, failing\n\tat $line11.$read$$iwC$$iwC$$iwC$$iwC$$anonfun$1.apply(:18)\n\tat $line11.$read$$iwC$$iwC$$iwC$$iwC$$anonfun$1.apply(:17)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:328)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1311)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:910)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:910)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$4.apply(SparkContext.scala:1314)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$4.apply(SparkContext.scala:1314)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:61)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:56)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:196)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)\n\tat java.lang.Thread.run(Thread.java:745)\n\nDriver stacktrace:","Stack Trace":[{"Declaring Class":"org.apache.spark.scheduler.DAGScheduler","Method Name":"org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages","File Name":"DAGScheduler.scala","Line Number":1214},{"Declaring Class":"org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1","Method Name":"apply","File Name":"DAGScheduler.scala","Line Number":1203},{"Declaring Class":"org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1","Method Name":"apply","File Name":"DAGScheduler.scala","Line Number":1202},{"Declaring Class":"scala.collection.mutable.ResizableArray$class","Method Name":"foreach","File Name":"ResizableArray.scala","Line Number":59},{"Declaring Class":"scala.collection.mutable.ArrayBuffer","Method Name":"foreach","File Name":"ArrayBuffer.scala","Line Number":47},{"Declaring Class":"org.apache.spark.scheduler.DAGScheduler","Method Name":"abortStage","File Name":"DAGScheduler.scala","Line Number":1202},{"Declaring Class":"org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1","Method Name":"apply","File Name":"DAGScheduler.scala","Line Number":696},{"Declaring Class":"org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1","Method Name":"apply","File Name":"DAGScheduler.scala","Line Number":696},{"Declaring Class":"scala.Option","Method Name":"foreach","File Name":"Option.scala","Line Number":236},{"Declaring Class":"org.apache.spark.scheduler.DAGScheduler","Method Name":"handleTaskSetFailed","File Name":"DAGScheduler.scala","Line Number":696},{"Declaring Class":"org.apache.spark.scheduler.DAGSchedulerEventProcessActor$$anonfun$receive$2","Method Name":"applyOrElse","File Name":"DAGScheduler.scala","Line Number":1420},{"Declaring Class":"akka.actor.Actor$class","Method Name":"aroundReceive","File Name":"Actor.scala","Line Number":465},{"Declaring Class":"org.apache.spark.scheduler.DAGSchedulerEventProcessActor","Method Name":"aroundReceive","File Name":"DAGScheduler.scala","Line Number":1375},{"Declaring Class":"akka.actor.ActorCell","Method Name":"receiveMessage","File Name":"ActorCell.scala","Line Number":516},{"Declaring Class":"akka.actor.ActorCell","Method Name":"invoke","File Name":"ActorCell.scala","Line Number":487},{"Declaring Class":"akka.dispatch.Mailbox","Method Name":"processMailbox","File Name":"Mailbox.scala","Line Number":238},{"Declaring Class":"akka.dispatch.Mailbox","Method Name":"run","File Name":"Mailbox.scala","Line Number":220},{"Declaring Class":"akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask","Method Name":"exec","File Name":"AbstractDispatcher.scala","Line Number":393},{"Declaring Class":"scala.concurrent.forkjoin.ForkJoinTask","Method Name":"doExec","File Name":"ForkJoinTask.java","Line Number":260},{"Declaring Class":"scala.concurrent.forkjoin.ForkJoinPool$WorkQueue","Method Name":"runTask","File Name":"ForkJoinPool.java","Line Number":1339},{"Declaring Class":"scala.concurrent.forkjoin.ForkJoinPool","Method Name":"runWorker","File Name":"ForkJoinPool.java","Line Number":1979},{"Declaring Class":"scala.concurrent.forkjoin.ForkJoinWorkerThread","Method Name":"run","File Name":"ForkJoinWorkerThread.java","Line Number":107}]}}} -{"Event":"SparkListenerJobStart","Job ID":2,"Stage Infos":[{"Stage ID":3,"Stage Attempt ID":0,"Stage Name":"count at :17","Number of Tasks":8,"RDD Info":[{"RDD ID":6,"Name":"/Users/irashid/spark-examples/tmp_data/sums_2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":5,"Name":"/Users/irashid/spark-examples/tmp_data/sums_2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line19.$read$$iwC$$iwC$$iwC$$iwC.(:17)\n$line19.$read$$iwC$$iwC$$iwC.(:22)\n$line19.$read$$iwC$$iwC.(:24)\n$line19.$read$$iwC.(:26)\n$line19.$read.(:28)\n$line19.$read$.(:32)\n$line19.$read$.()\n$line19.$eval$.(:7)\n$line19.$eval$.()\n$line19.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}],"Stage IDs":[3]} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":3,"Stage Attempt ID":0,"Stage Name":"count at :17","Number of Tasks":8,"RDD Info":[{"RDD ID":6,"Name":"/Users/irashid/spark-examples/tmp_data/sums_2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":5,"Name":"/Users/irashid/spark-examples/tmp_data/sums_2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line19.$read$$iwC$$iwC$$iwC$$iwC.(:17)\n$line19.$read$$iwC$$iwC$$iwC.(:22)\n$line19.$read$$iwC$$iwC.(:24)\n$line19.$read$$iwC.(:26)\n$line19.$read.(:28)\n$line19.$read$.(:32)\n$line19.$read$.()\n$line19.$eval$.(:7)\n$line19.$eval$.()\n$line19.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":3,"Stage Attempt ID":0,"Task Info":{"Task ID":24,"Index":0,"Attempt":0,"Launch Time":1422981787191,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":3,"Stage Attempt ID":0,"Task Info":{"Task ID":25,"Index":1,"Attempt":0,"Launch Time":1422981787191,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":3,"Stage Attempt ID":0,"Task Info":{"Task ID":26,"Index":2,"Attempt":0,"Launch Time":1422981787192,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":3,"Stage Attempt ID":0,"Task Info":{"Task ID":27,"Index":3,"Attempt":0,"Launch Time":1422981787192,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":3,"Stage Attempt ID":0,"Task Info":{"Task ID":28,"Index":4,"Attempt":0,"Launch Time":1422981787193,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":3,"Stage Attempt ID":0,"Task Info":{"Task ID":29,"Index":5,"Attempt":0,"Launch Time":1422981787193,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":3,"Stage Attempt ID":0,"Task Info":{"Task ID":30,"Index":6,"Attempt":0,"Launch Time":1422981787194,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":3,"Stage Attempt ID":0,"Task Info":{"Task ID":31,"Index":7,"Attempt":0,"Launch Time":1422981787194,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":3,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":25,"Index":1,"Attempt":0,"Launch Time":1422981787191,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981787222,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":5,"Executor Run Time":20,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":32}}} -{"Event":"SparkListenerTaskEnd","Stage ID":3,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":24,"Index":0,"Attempt":0,"Launch Time":1422981787191,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981787223,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":5,"Executor Run Time":20,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":32}}} -{"Event":"SparkListenerTaskEnd","Stage ID":3,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":26,"Index":2,"Attempt":0,"Launch Time":1422981787192,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981787223,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":22,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} -{"Event":"SparkListenerTaskEnd","Stage ID":3,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":31,"Index":7,"Attempt":0,"Launch Time":1422981787194,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981787223,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":5,"Executor Run Time":19,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} -{"Event":"SparkListenerTaskEnd","Stage ID":3,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":29,"Index":5,"Attempt":0,"Launch Time":1422981787193,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981787224,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":5,"Executor Run Time":19,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} -{"Event":"SparkListenerTaskEnd","Stage ID":3,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":27,"Index":3,"Attempt":0,"Launch Time":1422981787192,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981787224,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":6,"Executor Run Time":19,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} -{"Event":"SparkListenerTaskEnd","Stage ID":3,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":28,"Index":4,"Attempt":0,"Launch Time":1422981787193,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981787225,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":22,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} -{"Event":"SparkListenerTaskEnd","Stage ID":3,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":30,"Index":6,"Attempt":0,"Launch Time":1422981787194,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981787225,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":21,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":3,"Stage Attempt ID":0,"Stage Name":"count at :17","Number of Tasks":8,"RDD Info":[{"RDD ID":6,"Name":"/Users/irashid/spark-examples/tmp_data/sums_2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":5,"Name":"/Users/irashid/spark-examples/tmp_data/sums_2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line19.$read$$iwC$$iwC$$iwC$$iwC.(:17)\n$line19.$read$$iwC$$iwC$$iwC.(:22)\n$line19.$read$$iwC$$iwC.(:24)\n$line19.$read$$iwC.(:26)\n$line19.$read.(:28)\n$line19.$read$.(:32)\n$line19.$read$.()\n$line19.$eval$.(:7)\n$line19.$eval$.()\n$line19.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1422981787191,"Completion Time":1422981787226,"Accumulables":[]}} -{"Event":"SparkListenerJobEnd","Job ID":2,"Job Result":{"Result":"JobSucceeded"}} -{"Event":"SparkListenerApplicationEnd","Timestamp":1422981788731} diff --git a/core/src/test/resources/spark-events/local-1422981780767/SPARK_VERSION_1.2.0 b/core/src/test/resources/spark-events/local-1422981780767/SPARK_VERSION_1.2.0 deleted file mode 100755 index e69de29bb2d1d..0000000000000 diff --git a/core/src/test/resources/spark-events/local-1425081759269/APPLICATION_COMPLETE b/core/src/test/resources/spark-events/local-1425081759269/APPLICATION_COMPLETE deleted file mode 100755 index e69de29bb2d1d..0000000000000 diff --git a/core/src/test/resources/spark-events/local-1425081759269/EVENT_LOG_1 b/core/src/test/resources/spark-events/local-1425081759269/EVENT_LOG_1 deleted file mode 100755 index 9745b36b09e44..0000000000000 --- a/core/src/test/resources/spark-events/local-1425081759269/EVENT_LOG_1 +++ /dev/null @@ -1,88 +0,0 @@ -{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"","Host":"localhost","Port":57967},"Maximum Memory":278302556,"Timestamp":1425081759407} -{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre","Java Version":"1.7.0_67 (Oracle Corporation)","Scala Version":"version 2.10.4"},"Spark Properties":{"spark.driver.host":"192.168.1.103","spark.eventLog.enabled":"true","spark.driver.port":"57965","spark.repl.class.uri":"http://192.168.1.103:57964","spark.jars":"","spark.app.name":"Spark shell","spark.scheduler.mode":"FIFO","spark.executor.id":"driver","spark.master":"local[*]","spark.fileserver.uri":"http://192.168.1.103:57966","spark.tachyonStore.folderName":"spark-fd6c823a-8a18-4113-8306-1fa7bb623a7f","spark.app.id":"local-1425081759269"},"System Properties":{"java.io.tmpdir":"/var/folders/36/m29jw1z95qv4ywb1c4n0rz000000gp/T/","line.separator":"\n","path.separator":":","sun.management.compiler":"HotSpot 64-Bit Tiered Compilers","SPARK_SUBMIT":"true","sun.cpu.endian":"little","java.specification.version":"1.7","java.vm.specification.name":"Java Virtual Machine Specification","java.vendor":"Oracle Corporation","java.vm.specification.version":"1.7","user.home":"/Users/irashid","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","ftp.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","sun.arch.data.model":"64","sun.boot.library.path":"/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib","user.dir":"/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4","java.library.path":"/Users/irashid/Library/Java/Extensions:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java:.","sun.cpu.isalist":"","os.arch":"x86_64","java.vm.version":"24.65-b04","java.endorsed.dirs":"/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/endorsed","java.runtime.version":"1.7.0_67-b01","java.vm.info":"mixed mode","java.ext.dirs":"/Users/irashid/Library/Java/Extensions:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/ext:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java","java.runtime.name":"Java(TM) SE Runtime Environment","file.separator":"/","java.class.version":"51.0","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/resources.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/rt.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/sunrsasign.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/jsse.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/jce.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/charsets.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/jfr.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/classes","file.encoding":"UTF-8","user.timezone":"America/Chicago","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"10.9.5","sun.os.patch.level":"unknown","gopherProxySet":"false","java.vm.specification.vendor":"Oracle Corporation","user.country":"US","sun.jnu.encoding":"UTF-8","http.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","user.language":"en","socksNonProxyHosts":"local|*.local|169.254/16|*.169.254/16","java.vendor.url":"http://java.oracle.com/","java.awt.printerjob":"sun.lwawt.macosx.CPrinterJob","java.awt.graphicsenv":"sun.awt.CGraphicsEnvironment","awt.toolkit":"sun.lwawt.macosx.LWCToolkit","os.name":"Mac OS X","java.vm.vendor":"Oracle Corporation","java.vendor.url.bug":"http://bugreport.sun.com/bugreport/","user.name":"irashid","java.vm.name":"Java HotSpot(TM) 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --class org.apache.spark.repl.Main --conf spark.eventLog.enabled=true spark-shell","java.home":"/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre","java.version":"1.7.0_67","sun.io.unicode.encoding":"UnicodeBig"},"Classpath Entries":{"/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/spark-assembly-1.2.0-hadoop2.4.0.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-api-jdo-3.2.6.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-rdbms-3.2.9.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-core-3.2.10.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/conf":"System Classpath"}} -{"Event":"SparkListenerApplicationStart","App Name":"Spark shell","App ID":"local-1425081759269","Timestamp":1425081758277,"User":"irashid"} -{"Event":"SparkListenerJobStart","Job ID":0,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"count at :15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}],"Stage IDs":[0]} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"count at :15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1425081762075,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1425081762081,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1425081762081,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1425081762082,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1425081762083,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1425081762084,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1425081762084,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1425081762085,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1425081762084,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081762632,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":521,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_6","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1425081762081,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081762633,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":520,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_2","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1425081762082,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081762634,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":521,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_3","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1425081762084,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081762634,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":522,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_5","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1425081762083,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081762635,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":522,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_4","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1425081762075,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081762636,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":11,"Executor Run Time":522,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1425081762085,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081762636,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":9,"Executor Run Time":521,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_7","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1425081762081,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081762637,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":11,"Executor Run Time":522,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_1","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"count at :15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":8,"Memory Size":28000128,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1425081762069,"Completion Time":1425081762637,"Accumulables":[]}} -{"Event":"SparkListenerJobEnd","Job ID":0,"Job Result":{"Result":"JobSucceeded"}} -{"Event":"SparkListenerJobStart","Job ID":1,"Stage Infos":[{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"map at :14","Number of Tasks":8,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.(:14)\n$line10.$read$$iwC$$iwC$$iwC.(:19)\n$line10.$read$$iwC$$iwC.(:21)\n$line10.$read$$iwC.(:23)\n$line10.$read.(:25)\n$line10.$read$.(:29)\n$line10.$read$.()\n$line10.$eval$.(:7)\n$line10.$eval$.()\n$line10.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]},{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"first at :17","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.first(RDD.scala:1093)\n$line11.$read$$iwC$$iwC$$iwC$$iwC.(:17)\n$line11.$read$$iwC$$iwC$$iwC.(:22)\n$line11.$read$$iwC$$iwC.(:24)\n$line11.$read$$iwC.(:26)\n$line11.$read.(:28)\n$line11.$read$.(:32)\n$line11.$read$.()\n$line11.$eval$.(:7)\n$line11.$eval$.()\n$line11.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}],"Stage IDs":[1,2]} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"map at :14","Number of Tasks":8,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.(:14)\n$line10.$read$$iwC$$iwC$$iwC.(:19)\n$line10.$read$$iwC$$iwC.(:21)\n$line10.$read$$iwC.(:23)\n$line10.$read.(:25)\n$line10.$read$.(:29)\n$line10.$read$.()\n$line10.$eval$.(:7)\n$line10.$eval$.()\n$line10.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":8,"Index":0,"Attempt":0,"Launch Time":1425081763578,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":9,"Index":1,"Attempt":0,"Launch Time":1425081763578,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":10,"Index":2,"Attempt":0,"Launch Time":1425081763579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":11,"Index":3,"Attempt":0,"Launch Time":1425081763579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":12,"Index":4,"Attempt":0,"Launch Time":1425081763580,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":13,"Index":5,"Attempt":0,"Launch Time":1425081763580,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":14,"Index":6,"Attempt":0,"Launch Time":1425081763581,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":15,"Index":7,"Attempt":0,"Launch Time":1425081763581,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":10,"Index":2,"Attempt":0,"Launch Time":1425081763579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764001,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":406,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":138000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} -{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":9,"Index":1,"Attempt":0,"Launch Time":1425081763578,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764002,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":407,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":106000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} -{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":12,"Index":4,"Attempt":0,"Launch Time":1425081763580,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764002,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":407,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1645,"Shuffle Write Time":99000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} -{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":8,"Index":0,"Attempt":0,"Launch Time":1425081763578,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764003,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":407,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":123000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} -{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":15,"Index":7,"Attempt":0,"Launch Time":1425081763581,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764003,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":406,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":108000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} -{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":11,"Index":3,"Attempt":0,"Launch Time":1425081763579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764004,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":407,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1647,"Shuffle Write Time":97000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} -{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":14,"Index":6,"Attempt":0,"Launch Time":1425081763581,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764004,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":407,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":132000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} -{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":13,"Index":5,"Attempt":0,"Launch Time":1425081763580,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764005,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":407,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":81000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"map at :14","Number of Tasks":8,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.(:14)\n$line10.$read$$iwC$$iwC$$iwC.(:19)\n$line10.$read$$iwC$$iwC.(:21)\n$line10.$read$$iwC.(:23)\n$line10.$read.(:25)\n$line10.$read$.(:29)\n$line10.$read$.()\n$line10.$eval$.(:7)\n$line10.$eval$.()\n$line10.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1425081763578,"Completion Time":1425081764005,"Accumulables":[]}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"first at :17","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.first(RDD.scala:1093)\n$line11.$read$$iwC$$iwC$$iwC$$iwC.(:17)\n$line11.$read$$iwC$$iwC$$iwC.(:22)\n$line11.$read$$iwC$$iwC.(:24)\n$line11.$read$$iwC.(:26)\n$line11.$read.(:28)\n$line11.$read$.(:32)\n$line11.$read$.()\n$line11.$eval$.(:7)\n$line11.$eval$.()\n$line11.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":16,"Index":0,"Attempt":0,"Launch Time":1425081764014,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":16,"Index":0,"Attempt":0,"Launch Time":1425081764014,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764045,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":28,"Result Size":1013,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"first at :17","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.first(RDD.scala:1093)\n$line11.$read$$iwC$$iwC$$iwC$$iwC.(:17)\n$line11.$read$$iwC$$iwC$$iwC.(:22)\n$line11.$read$$iwC$$iwC.(:24)\n$line11.$read$$iwC.(:26)\n$line11.$read.(:28)\n$line11.$read$.(:32)\n$line11.$read$.()\n$line11.$eval$.(:7)\n$line11.$eval$.()\n$line11.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1425081764014,"Completion Time":1425081764045,"Accumulables":[]}} -{"Event":"SparkListenerJobEnd","Job ID":1,"Job Result":{"Result":"JobSucceeded"}} -{"Event":"SparkListenerJobStart","Job ID":2,"Stage Infos":[{"Stage ID":3,"Stage Attempt ID":0,"Stage Name":"map at :14","Number of Tasks":8,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.(:14)\n$line10.$read$$iwC$$iwC$$iwC.(:19)\n$line10.$read$$iwC$$iwC.(:21)\n$line10.$read$$iwC.(:23)\n$line10.$read.(:25)\n$line10.$read$.(:29)\n$line10.$read$.()\n$line10.$eval$.(:7)\n$line10.$eval$.()\n$line10.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]},{"Stage ID":4,"Stage Attempt ID":0,"Stage Name":"saveAsTextFile at :19","Number of Tasks":8,"RDD Info":[{"RDD ID":3,"Name":"3","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.saveAsTextFile(RDD.scala:1164)\n$line13.$read$$iwC$$iwC$$iwC$$iwC.(:19)\n$line13.$read$$iwC$$iwC$$iwC.(:24)\n$line13.$read$$iwC$$iwC.(:26)\n$line13.$read$$iwC.(:28)\n$line13.$read.(:30)\n$line13.$read$.(:34)\n$line13.$read$.()\n$line13.$eval$.(:7)\n$line13.$eval$.()\n$line13.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}],"Stage IDs":[3,4]} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":4,"Stage Attempt ID":0,"Stage Name":"saveAsTextFile at :19","Number of Tasks":8,"RDD Info":[{"RDD ID":3,"Name":"3","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.saveAsTextFile(RDD.scala:1164)\n$line13.$read$$iwC$$iwC$$iwC$$iwC.(:19)\n$line13.$read$$iwC$$iwC$$iwC.(:24)\n$line13.$read$$iwC$$iwC.(:26)\n$line13.$read$$iwC.(:28)\n$line13.$read.(:30)\n$line13.$read$.(:34)\n$line13.$read$.()\n$line13.$eval$.(:7)\n$line13.$eval$.()\n$line13.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":17,"Index":0,"Attempt":0,"Launch Time":1425081764396,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":18,"Index":1,"Attempt":0,"Launch Time":1425081764396,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":19,"Index":2,"Attempt":0,"Launch Time":1425081764397,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":20,"Index":3,"Attempt":0,"Launch Time":1425081764397,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":21,"Index":4,"Attempt":0,"Launch Time":1425081764398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":22,"Index":5,"Attempt":0,"Launch Time":1425081764398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":23,"Index":6,"Attempt":0,"Launch Time":1425081764398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":24,"Index":7,"Attempt":0,"Launch Time":1425081764399,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":24,"Index":7,"Attempt":0,"Launch Time":1425081764399,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764642,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":240,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":19,"Index":2,"Attempt":0,"Launch Time":1425081764397,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764643,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":241,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":21,"Index":4,"Attempt":0,"Launch Time":1425081764398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764643,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":240,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":22,"Index":5,"Attempt":0,"Launch Time":1425081764398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764645,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":241,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":23,"Index":6,"Attempt":0,"Launch Time":1425081764398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764645,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":241,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":17,"Index":0,"Attempt":0,"Launch Time":1425081764396,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764646,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":243,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":18,"Index":1,"Attempt":0,"Launch Time":1425081764396,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764646,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":243,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":20,"Index":3,"Attempt":0,"Launch Time":1425081764397,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764648,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":247,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":4,"Stage Attempt ID":0,"Stage Name":"saveAsTextFile at :19","Number of Tasks":8,"RDD Info":[{"RDD ID":3,"Name":"3","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.saveAsTextFile(RDD.scala:1164)\n$line13.$read$$iwC$$iwC$$iwC$$iwC.(:19)\n$line13.$read$$iwC$$iwC$$iwC.(:24)\n$line13.$read$$iwC$$iwC.(:26)\n$line13.$read$$iwC.(:28)\n$line13.$read.(:30)\n$line13.$read$.(:34)\n$line13.$read$.()\n$line13.$eval$.(:7)\n$line13.$eval$.()\n$line13.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1425081764396,"Completion Time":1425081764648,"Accumulables":[]}} -{"Event":"SparkListenerJobEnd","Job ID":2,"Job Result":{"Result":"JobSucceeded"}} -{"Event":"SparkListenerJobStart","Job ID":3,"Stage Infos":[{"Stage ID":5,"Stage Attempt ID":0,"Stage Name":"count at :17","Number of Tasks":8,"RDD Info":[{"RDD ID":5,"Name":"/Users/irashid/spark-examples/tmp_data/sums","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":4,"Name":"/Users/irashid/spark-examples/tmp_data/sums","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.(:17)\n$line15.$read$$iwC$$iwC$$iwC.(:22)\n$line15.$read$$iwC$$iwC.(:24)\n$line15.$read$$iwC.(:26)\n$line15.$read.(:28)\n$line15.$read$.(:32)\n$line15.$read$.()\n$line15.$eval$.(:7)\n$line15.$eval$.()\n$line15.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}],"Stage IDs":[5]} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":5,"Stage Attempt ID":0,"Stage Name":"count at :17","Number of Tasks":8,"RDD Info":[{"RDD ID":5,"Name":"/Users/irashid/spark-examples/tmp_data/sums","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":4,"Name":"/Users/irashid/spark-examples/tmp_data/sums","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.(:17)\n$line15.$read$$iwC$$iwC$$iwC.(:22)\n$line15.$read$$iwC$$iwC.(:24)\n$line15.$read$$iwC.(:26)\n$line15.$read.(:28)\n$line15.$read$.(:32)\n$line15.$read$.()\n$line15.$eval$.(:7)\n$line15.$eval$.()\n$line15.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":25,"Index":0,"Attempt":0,"Launch Time":1425081765026,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":26,"Index":1,"Attempt":0,"Launch Time":1425081765026,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":27,"Index":2,"Attempt":0,"Launch Time":1425081765027,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":28,"Index":3,"Attempt":0,"Launch Time":1425081765027,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":29,"Index":4,"Attempt":0,"Launch Time":1425081765027,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":30,"Index":5,"Attempt":0,"Launch Time":1425081765028,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":31,"Index":6,"Attempt":0,"Launch Time":1425081765028,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":32,"Index":7,"Attempt":0,"Launch Time":1425081765029,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":27,"Index":2,"Attempt":0,"Launch Time":1425081765027,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081765045,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} -{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":30,"Index":5,"Attempt":0,"Launch Time":1425081765028,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081765046,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} -{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":25,"Index":0,"Attempt":0,"Launch Time":1425081765026,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081765046,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":12,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":32}}} -{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":29,"Index":4,"Attempt":0,"Launch Time":1425081765027,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081765047,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} -{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":28,"Index":3,"Attempt":0,"Launch Time":1425081765027,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081765047,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":12,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} -{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":32,"Index":7,"Attempt":0,"Launch Time":1425081765029,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081765048,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} -{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":31,"Index":6,"Attempt":0,"Launch Time":1425081765028,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081765048,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":14,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} -{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":26,"Index":1,"Attempt":0,"Launch Time":1425081765026,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081765049,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":12,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":32}}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":5,"Stage Attempt ID":0,"Stage Name":"count at :17","Number of Tasks":8,"RDD Info":[{"RDD ID":5,"Name":"/Users/irashid/spark-examples/tmp_data/sums","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":4,"Name":"/Users/irashid/spark-examples/tmp_data/sums","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.(:17)\n$line15.$read$$iwC$$iwC$$iwC.(:22)\n$line15.$read$$iwC$$iwC.(:24)\n$line15.$read$$iwC.(:26)\n$line15.$read.(:28)\n$line15.$read$.(:32)\n$line15.$read$.()\n$line15.$eval$.(:7)\n$line15.$eval$.()\n$line15.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1425081765026,"Completion Time":1425081765050,"Accumulables":[]}} -{"Event":"SparkListenerJobEnd","Job ID":3,"Job Result":{"Result":"JobSucceeded"}} -{"Event":"SparkListenerApplicationEnd","Timestamp":1425081766912} diff --git a/core/src/test/resources/spark-events/local-1425081759269/SPARK_VERSION_1.2.0 b/core/src/test/resources/spark-events/local-1425081759269/SPARK_VERSION_1.2.0 deleted file mode 100755 index e69de29bb2d1d..0000000000000 diff --git a/core/src/test/resources/spark-events/local-1426533911241/APPLICATION_COMPLETE b/core/src/test/resources/spark-events/local-1426533911241/APPLICATION_COMPLETE deleted file mode 100755 index e69de29bb2d1d..0000000000000 diff --git a/core/src/test/resources/spark-events/local-1426533911241/EVENT_LOG_1 b/core/src/test/resources/spark-events/local-1426533911241/EVENT_LOG_1 deleted file mode 100755 index 9ef5bd5d92de5..0000000000000 --- a/core/src/test/resources/spark-events/local-1426533911241/EVENT_LOG_1 +++ /dev/null @@ -1,24 +0,0 @@ -{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"","Host":"localhost","Port":58610},"Maximum Memory":278019440,"Timestamp":1426533911361} -{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre","Java Version":"1.8.0_25 (Oracle Corporation)","Scala Version":"version 2.10.4"},"Spark Properties":{"spark.driver.host":"192.168.1.105","spark.eventLog.enabled":"true","spark.driver.port":"58608","spark.repl.class.uri":"http://192.168.1.105:58607","spark.jars":"","spark.app.name":"Spark shell","spark.scheduler.mode":"FIFO","spark.executor.id":"driver","spark.master":"local[*]","spark.fileserver.uri":"http://192.168.1.105:58609","spark.tachyonStore.folderName":"spark-5e9b7f26-8e97-4b43-82d6-25c141530da9","spark.app.id":"local-1426533911241"},"System Properties":{"java.io.tmpdir":"/var/folders/36/m29jw1z95qv4ywb1c4n0rz000000gp/T/","line.separator":"\n","path.separator":":","sun.management.compiler":"HotSpot 64-Bit Tiered Compilers","SPARK_SUBMIT":"true","sun.cpu.endian":"little","java.specification.version":"1.8","java.vm.specification.name":"Java Virtual Machine Specification","java.vendor":"Oracle Corporation","java.vm.specification.version":"1.8","user.home":"/Users/irashid","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","ftp.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","sun.arch.data.model":"64","sun.boot.library.path":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib","user.dir":"/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4","java.library.path":"/Users/irashid/Library/Java/Extensions:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java:.","sun.cpu.isalist":"","os.arch":"x86_64","java.vm.version":"25.25-b02","java.endorsed.dirs":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/endorsed","java.runtime.version":"1.8.0_25-b17","java.vm.info":"mixed mode","java.ext.dirs":"/Users/irashid/Library/Java/Extensions:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/ext:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java","java.runtime.name":"Java(TM) SE Runtime Environment","file.separator":"/","java.class.version":"52.0","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/resources.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/rt.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/sunrsasign.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/jsse.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/jce.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/charsets.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/jfr.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/classes","file.encoding":"UTF-8","user.timezone":"America/Chicago","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"10.9.5","sun.os.patch.level":"unknown","gopherProxySet":"false","java.vm.specification.vendor":"Oracle Corporation","user.country":"US","sun.jnu.encoding":"UTF-8","http.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","user.language":"en","socksNonProxyHosts":"local|*.local|169.254/16|*.169.254/16","java.vendor.url":"http://java.oracle.com/","java.awt.printerjob":"sun.lwawt.macosx.CPrinterJob","java.awt.graphicsenv":"sun.awt.CGraphicsEnvironment","awt.toolkit":"sun.lwawt.macosx.LWCToolkit","os.name":"Mac OS X","java.vm.vendor":"Oracle Corporation","java.vendor.url.bug":"http://bugreport.sun.com/bugreport/","user.name":"irashid","java.vm.name":"Java HotSpot(TM) 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --class org.apache.spark.repl.Main --conf spark.eventLog.enabled=true spark-shell","java.home":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre","java.version":"1.8.0_25","sun.io.unicode.encoding":"UnicodeBig"},"Classpath Entries":{"/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/spark-assembly-1.2.0-hadoop2.4.0.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-api-jdo-3.2.6.jar":"System Classpath","/etc/hadoop":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-rdbms-3.2.9.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-core-3.2.10.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/conf":"System Classpath"}} -{"Event":"SparkListenerApplicationStart","App Name":"Spark shell","App ID":"local-1426533911241","Timestamp":1426533910242,"User":"irashid","App Attempt ID":"1"} -{"Event":"SparkListenerJobStart","Job ID":0,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"foreach at :15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.foreach(RDD.scala:765)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}],"Stage IDs":[0]} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"foreach at :15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.foreach(RDD.scala:765)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1426533936515,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1426533936521,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1426533936522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1426533936522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1426533936522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1426533936523,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1426533936523,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1426533936524,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1426533936522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426533936570,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"378","Value":"378"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":13,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1426533936522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426533936572,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"572","Value":"950"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":13,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1426533936523,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426533936574,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"978","Value":"1928"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1426533936521,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426533936574,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"247","Value":"2175"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":14,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1426533936522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426533936574,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"678","Value":"2853"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1426533936523,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426533936575,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"897","Value":"3750"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1426533936524,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426533936575,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"1222","Value":"4972"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1426533936515,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426533936576,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"78","Value":"5050"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":14,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"foreach at :15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.foreach(RDD.scala:765)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1426533936103,"Completion Time":1426533936579,"Accumulables":[{"ID":1,"Name":"my counter","Value":"5050"}]}} -{"Event":"SparkListenerJobEnd","Job ID":0,"Job Result":{"Result":"JobSucceeded"}} -{"Event":"SparkListenerApplicationEnd","Timestamp":1426533945177} diff --git a/core/src/test/resources/spark-events/local-1426533911241/SPARK_VERSION_1.2.0 b/core/src/test/resources/spark-events/local-1426533911241/SPARK_VERSION_1.2.0 deleted file mode 100755 index e69de29bb2d1d..0000000000000 diff --git a/core/src/test/resources/spark-events/local-1426633911242/APPLICATION_COMPLETE b/core/src/test/resources/spark-events/local-1426633911242/APPLICATION_COMPLETE deleted file mode 100755 index e69de29bb2d1d..0000000000000 diff --git a/core/src/test/resources/spark-events/local-1426633911242/EVENT_LOG_1 b/core/src/test/resources/spark-events/local-1426633911242/EVENT_LOG_1 deleted file mode 100755 index e7043282107d3..0000000000000 --- a/core/src/test/resources/spark-events/local-1426633911242/EVENT_LOG_1 +++ /dev/null @@ -1,24 +0,0 @@ -{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"","Host":"localhost","Port":58610},"Maximum Memory":278019440,"Timestamp":1426633911361} -{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre","Java Version":"1.8.0_25 (Oracle Corporation)","Scala Version":"version 2.10.4"},"Spark Properties":{"spark.driver.host":"192.168.1.105","spark.eventLog.enabled":"true","spark.driver.port":"58608","spark.repl.class.uri":"http://192.168.1.105:58607","spark.jars":"","spark.app.name":"Spark shell","spark.scheduler.mode":"FIFO","spark.executor.id":"driver","spark.master":"local[*]","spark.fileserver.uri":"http://192.168.1.105:58609","spark.tachyonStore.folderName":"spark-5e9b7f26-8e97-4b43-82d6-25c141530da9","spark.app.id":"local-1426633911241"},"System Properties":{"java.io.tmpdir":"/var/folders/36/m29jw1z95qv4ywb1c4n0rz000000gp/T/","line.separator":"\n","path.separator":":","sun.management.compiler":"HotSpot 64-Bit Tiered Compilers","SPARK_SUBMIT":"true","sun.cpu.endian":"little","java.specification.version":"1.8","java.vm.specification.name":"Java Virtual Machine Specification","java.vendor":"Oracle Corporation","java.vm.specification.version":"1.8","user.home":"/Users/irashid","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","ftp.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","sun.arch.data.model":"64","sun.boot.library.path":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib","user.dir":"/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4","java.library.path":"/Users/irashid/Library/Java/Extensions:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java:.","sun.cpu.isalist":"","os.arch":"x86_64","java.vm.version":"25.25-b02","java.endorsed.dirs":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/endorsed","java.runtime.version":"1.8.0_25-b17","java.vm.info":"mixed mode","java.ext.dirs":"/Users/irashid/Library/Java/Extensions:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/ext:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java","java.runtime.name":"Java(TM) SE Runtime Environment","file.separator":"/","java.class.version":"52.0","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/resources.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/rt.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/sunrsasign.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/jsse.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/jce.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/charsets.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/jfr.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/classes","file.encoding":"UTF-8","user.timezone":"America/Chicago","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"10.9.5","sun.os.patch.level":"unknown","gopherProxySet":"false","java.vm.specification.vendor":"Oracle Corporation","user.country":"US","sun.jnu.encoding":"UTF-8","http.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","user.language":"en","socksNonProxyHosts":"local|*.local|169.254/16|*.169.254/16","java.vendor.url":"http://java.oracle.com/","java.awt.printerjob":"sun.lwawt.macosx.CPrinterJob","java.awt.graphicsenv":"sun.awt.CGraphicsEnvironment","awt.toolkit":"sun.lwawt.macosx.LWCToolkit","os.name":"Mac OS X","java.vm.vendor":"Oracle Corporation","java.vendor.url.bug":"http://bugreport.sun.com/bugreport/","user.name":"irashid","java.vm.name":"Java HotSpot(TM) 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --class org.apache.spark.repl.Main --conf spark.eventLog.enabled=true spark-shell","java.home":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre","java.version":"1.8.0_25","sun.io.unicode.encoding":"UnicodeBig"},"Classpath Entries":{"/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/spark-assembly-1.2.0-hadoop2.4.0.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-api-jdo-3.2.6.jar":"System Classpath","/etc/hadoop":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-rdbms-3.2.9.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-core-3.2.10.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/conf":"System Classpath"}} -{"Event":"SparkListenerApplicationStart","App Name":"Spark shell","App ID":"local-1426533911241","Timestamp":1426633910242,"User":"irashid","App Attempt ID":"2"} -{"Event":"SparkListenerJobStart","Job ID":0,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"foreach at :15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.foreach(RDD.scala:765)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}],"Stage IDs":[0]} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"foreach at :15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.foreach(RDD.scala:765)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1426633936515,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1426633936521,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1426633936522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1426633936522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1426633936522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1426633936523,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1426633936523,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1426633936524,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1426633936522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426633936570,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"378","Value":"378"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":13,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1426633936522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426633936572,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"572","Value":"950"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":13,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1426633936523,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426633936574,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"978","Value":"1928"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1426633936521,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426633936574,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"247","Value":"2175"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":14,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1426633936522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426633936574,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"678","Value":"2853"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1426633936523,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426633936575,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"897","Value":"3750"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1426633936524,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426633936575,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"1222","Value":"4972"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1426633936515,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426633936576,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"78","Value":"5050"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":14,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"foreach at :15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.foreach(RDD.scala:765)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1426633936103,"Completion Time":1426633936579,"Accumulables":[{"ID":1,"Name":"my counter","Value":"5050"}]}} -{"Event":"SparkListenerJobEnd","Job ID":0,"Job Result":{"Result":"JobSucceeded"}} -{"Event":"SparkListenerApplicationEnd","Timestamp":1426633945177} diff --git a/core/src/test/resources/spark-events/local-1426633911242/SPARK_VERSION_1.2.0 b/core/src/test/resources/spark-events/local-1426633911242/SPARK_VERSION_1.2.0 deleted file mode 100755 index e69de29bb2d1d..0000000000000 diff --git a/core/src/test/resources/spark-events/local-1427397477963 b/core/src/test/resources/spark-events/local-1427397477963 deleted file mode 100755 index 3bd251d48c87b..0000000000000 --- a/core/src/test/resources/spark-events/local-1427397477963 +++ /dev/null @@ -1,12083 +0,0 @@ -{"Event":"SparkListenerLogStart","Spark Version":"1.4.0-SNAPSHOT"} -{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"","Host":"localhost","Port":53827},"Maximum Memory":257918238,"Timestamp":1427397478113} -{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre","Java Version":"1.8.0_25 (Oracle Corporation)","Scala Version":"version 2.10.4"},"Spark Properties":{"spark.driver.host":"192.168.1.101","spark.eventLog.enabled":"true","spark.driver.port":"53825","spark.repl.class.uri":"http://192.168.1.101:53823","spark.jars":"","spark.app.name":"Spark shell","spark.scheduler.mode":"FIFO","spark.cleaner.ttl.MAP_OUTPUT_TRACKER":"1","spark.executor.id":"","spark.master":"local[*]","spark.fileserver.uri":"http://192.168.1.101:53826","spark.tachyonStore.folderName":"spark-ba9af2c0-12a3-4d07-8f0a-2aded3ba3ded","spark.app.id":"local-1427397477963"},"System Properties":{"java.io.tmpdir":"/var/folders/36/m29jw1z95qv4ywb1c4n0rz000000gp/T/","line.separator":"\n","path.separator":":","sun.management.compiler":"HotSpot 64-Bit Tiered Compilers","SPARK_SUBMIT":"true","sun.cpu.endian":"little","java.specification.version":"1.8","java.vm.specification.name":"Java Virtual Machine Specification","java.vendor":"Oracle Corporation","java.vm.specification.version":"1.8","user.home":"/Users/irashid","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","ftp.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","sun.arch.data.model":"64","sun.boot.library.path":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib","user.dir":"/Users/irashid/github/spark","java.library.path":"/Users/irashid/Library/Java/Extensions:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java:.","sun.cpu.isalist":"","os.arch":"x86_64","java.vm.version":"25.25-b02","java.endorsed.dirs":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/endorsed","java.runtime.version":"1.8.0_25-b17","java.vm.info":"mixed mode","java.ext.dirs":"/Users/irashid/Library/Java/Extensions:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/ext:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java","java.runtime.name":"Java(TM) SE Runtime Environment","file.separator":"/","java.class.version":"52.0","scala.usejavacp":"true","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/resources.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/rt.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/sunrsasign.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/jsse.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/jce.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/charsets.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/jfr.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/classes","file.encoding":"UTF-8","user.timezone":"America/Chicago","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"10.9.5","sun.os.patch.level":"unknown","gopherProxySet":"false","java.vm.specification.vendor":"Oracle Corporation","user.country":"US","sun.jnu.encoding":"UTF-8","http.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","user.language":"en","socksNonProxyHosts":"local|*.local|169.254/16|*.169.254/16","java.vendor.url":"http://java.oracle.com/","java.awt.printerjob":"sun.lwawt.macosx.CPrinterJob","java.awt.graphicsenv":"sun.awt.CGraphicsEnvironment","awt.toolkit":"sun.lwawt.macosx.LWCToolkit","os.name":"Mac OS X","java.vm.vendor":"Oracle Corporation","java.vendor.url.bug":"http://bugreport.sun.com/bugreport/","user.name":"irashid","java.vm.name":"Java HotSpot(TM) 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --conf spark.eventLog.enabled=true --conf spark.cleaner.ttl.MAP_OUTPUT_TRACKER=1 --class org.apache.spark.repl.Main spark-shell","java.home":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre","java.version":"1.8.0_25","sun.io.unicode.encoding":"UnicodeBig"},"Classpath Entries":{"/Users/irashid/github/spark/tools/target/scala-2.10/classes/":"System Classpath","/Users/irashid/github/spark/core/target/jars/jetty-continuation-8.1.14.v20131031.jar":"System Classpath","/Users/irashid/github/spark/core/target/scala-2.10/classes/":"System Classpath","/Users/irashid/github/spark/yarn/target/scala-2.10/classes/":"System Classpath","/Users/irashid/github/spark/core/target/jars/jetty-http-8.1.14.v20131031.jar":"System Classpath","/etc/hadoop":"System Classpath","/Users/irashid/github/spark/lib_managed/jars/datanucleus-rdbms-3.2.9.jar":"System Classpath","/Users/irashid/github/spark/launcher/target/scala-2.10/classes/":"System Classpath","/Users/irashid/github/spark/streaming/target/scala-2.10/classes/":"System Classpath","/Users/irashid/github/spark/sql/hive/target/scala-2.10/classes/":"System Classpath","/Users/irashid/github/spark/core/target/jars/guava-14.0.1.jar":"System Classpath","/Users/irashid/github/spark/sql/core/target/scala-2.10/classes/":"System Classpath","/Users/irashid/github/spark/repl/target/scala-2.10/classes/":"System Classpath","/Users/irashid/github/spark/conf/":"System Classpath","/Users/irashid/github/spark/sql/hive-thriftserver/target/scala-2.10/classes/":"System Classpath","/Users/irashid/github/spark/bagel/target/scala-2.10/classes/":"System Classpath","/Users/irashid/github/spark/core/target/jars/jetty-security-8.1.14.v20131031.jar":"System Classpath","/Users/irashid/github/spark/core/target/jars/jetty-server-8.1.14.v20131031.jar":"System Classpath","/Users/irashid/github/spark/sql/catalyst/target/scala-2.10/classes/":"System Classpath","/Users/irashid/github/spark/core/target/jars/jetty-util-8.1.14.v20131031.jar":"System Classpath","/Users/irashid/github/spark/core/target/jars/jetty-servlet-8.1.14.v20131031.jar":"System Classpath","/Users/irashid/github/spark/core/target/jars/jetty-io-8.1.14.v20131031.jar":"System Classpath","/Users/irashid/github/spark/mllib/target/scala-2.10/classes/":"System Classpath","/Users/irashid/github/spark/assembly/target/scala-2.10/spark-assembly-1.4.0-SNAPSHOT-hadoop2.5.0.jar":"System Classpath","/Users/irashid/github/spark/core/target/jars/jetty-plus-8.1.14.v20131031.jar":"System Classpath","/Users/irashid/github/spark/lib_managed/jars/datanucleus-core-3.2.10.jar":"System Classpath","/Users/irashid/github/spark/lib_managed/jars/datanucleus-api-jdo-3.2.6.jar":"System Classpath","/Users/irashid/github/spark/graphx/target/scala-2.10/classes/":"System Classpath","/Users/irashid/github/spark/core/target/jars/jetty-util-6.1.26.jar":"System Classpath"}} -{"Event":"SparkListenerApplicationStart","App Name":"Spark shell","App ID":"local-1427397477963","Timestamp":1427397477184,"User":"irashid"} -{"Event":"SparkListenerJobStart","Job ID":0,"Submission Time":1427397514437,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"map at :21","Number of Tasks":1000,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:287)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:21)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:28)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:30)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.(:32)\n$line15.$read$$iwC$$iwC$$iwC.(:34)\n$line15.$read$$iwC$$iwC.(:36)\n$line15.$read$$iwC.(:38)\n$line15.$read.(:40)\n$line15.$read$.(:44)\n$line15.$read$.()\n$line15.$eval$.(:7)\n$line15.$eval$.()\n$line15.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]},{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"replStringOf at :10","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":50,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"scala.runtime.ScalaRunTime$.replStringOf(ScalaRunTime.scala:337)\n$line19.$eval$.(:10)\n$line19.$eval$.()\n$line19.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1338)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:840)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:871)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:819)\norg.apache.spark.repl.SparkILoop.reallyInterpret$1(SparkILoop.scala:856)\norg.apache.spark.repl.SparkILoop.interpretStartingWith(SparkILoop.scala:901)\norg.apache.spark.repl.SparkILoop.command(SparkILoop.scala:813)\norg.apache.spark.repl.SparkILoop.processLine$1(SparkILoop.scala:656)\norg.apache.spark.repl.SparkILoop.innerLoop$1(SparkILoop.scala:664)\norg.apache.spark.repl.SparkILoop.org$apache$spark$repl$SparkILoop$$loop(SparkILoop.scala:669)\norg.apache.spark.repl.SparkILoop$$anonfun$org$apache$spark$repl$SparkILoop$$process$1.apply$mcZ$sp(SparkILoop.scala:996)","Accumulables":[]}],"Stage IDs":[0,1]} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"map at :21","Number of Tasks":1000,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:287)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:21)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:28)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:30)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.(:32)\n$line15.$read$$iwC$$iwC$$iwC.(:34)\n$line15.$read$$iwC$$iwC.(:36)\n$line15.$read$$iwC.(:38)\n$line15.$read.(:40)\n$line15.$read$.(:44)\n$line15.$read$.()\n$line15.$eval$.(:7)\n$line15.$eval$.()\n$line15.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1427397514583,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1427397514599,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1427397514600,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1427397514600,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1427397514601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1427397514601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1427397514602,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1427397514602,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":8,"Index":8,"Attempt":0,"Launch Time":1427397514895,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":9,"Index":9,"Attempt":0,"Launch Time":1427397514897,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":10,"Index":10,"Attempt":0,"Launch Time":1427397514898,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1427397514602,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397514903,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":14,"Executor Run Time":262,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":2709542,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1427397514601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397514906,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":15,"Executor Run Time":262,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1683487,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":11,"Index":11,"Attempt":0,"Launch Time":1427397514907,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1427397514602,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397514908,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":13,"Executor Run Time":273,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":438240,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":12,"Index":12,"Attempt":0,"Launch Time":1427397514908,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1427397514599,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397514910,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":15,"Executor Run Time":262,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":2220286,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":13,"Index":13,"Attempt":0,"Launch Time":1427397514910,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1427397514600,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397514911,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":15,"Executor Run Time":270,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":1664784,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1427397514600,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397514911,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":16,"Executor Run Time":261,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":2178902,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":14,"Index":14,"Attempt":0,"Launch Time":1427397514912,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":15,"Index":15,"Attempt":0,"Launch Time":1427397514912,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1427397514583,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397514916,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":16,"Executor Run Time":274,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":4718634,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1427397514601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397514916,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":15,"Executor Run Time":269,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1786420,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":16,"Index":16,"Attempt":0,"Launch Time":1427397514975,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":13,"Index":13,"Attempt":0,"Launch Time":1427397514910,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397514975,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":59,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":430298,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":17,"Index":17,"Attempt":0,"Launch Time":1427397514979,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":12,"Index":12,"Attempt":0,"Launch Time":1427397514908,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397514979,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":63,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":419405,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":18,"Index":18,"Attempt":0,"Launch Time":1427397514989,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":14,"Index":14,"Attempt":0,"Launch Time":1427397514912,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397514989,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":73,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":433438,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":19,"Index":19,"Attempt":0,"Launch Time":1427397514995,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":10,"Index":10,"Attempt":0,"Launch Time":1427397514898,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397514996,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":91,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":479137,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":20,"Index":20,"Attempt":0,"Launch Time":1427397514997,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":15,"Index":15,"Attempt":0,"Launch Time":1427397514912,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397514998,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":79,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":478808,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":21,"Index":21,"Attempt":0,"Launch Time":1427397515002,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":8,"Index":8,"Attempt":0,"Launch Time":1427397514895,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515002,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":103,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":525454,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":22,"Index":22,"Attempt":0,"Launch Time":1427397515011,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":11,"Index":11,"Attempt":0,"Launch Time":1427397514907,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515012,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":101,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":593414,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":23,"Index":23,"Attempt":0,"Launch Time":1427397515012,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":9,"Index":9,"Attempt":0,"Launch Time":1427397514897,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515013,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":112,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":524832,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":24,"Index":24,"Attempt":0,"Launch Time":1427397515055,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":16,"Index":16,"Attempt":0,"Launch Time":1427397514975,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515056,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":78,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":504502,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":25,"Index":25,"Attempt":0,"Launch Time":1427397515059,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":22,"Index":22,"Attempt":0,"Launch Time":1427397515011,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515060,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":46,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":459758,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":26,"Index":26,"Attempt":0,"Launch Time":1427397515060,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":17,"Index":17,"Attempt":0,"Launch Time":1427397514979,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515060,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":80,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1349002,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":27,"Index":27,"Attempt":0,"Launch Time":1427397515062,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":18,"Index":18,"Attempt":0,"Launch Time":1427397514989,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515062,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":71,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":545262,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":28,"Index":28,"Attempt":0,"Launch Time":1427397515069,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":19,"Index":19,"Attempt":0,"Launch Time":1427397514995,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515070,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":72,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":561286,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":29,"Index":29,"Attempt":0,"Launch Time":1427397515074,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":20,"Index":20,"Attempt":0,"Launch Time":1427397514997,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515074,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":74,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":492272,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":30,"Index":30,"Attempt":0,"Launch Time":1427397515074,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":23,"Index":23,"Attempt":0,"Launch Time":1427397515012,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515075,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":59,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":511744,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":31,"Index":31,"Attempt":0,"Launch Time":1427397515082,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":21,"Index":21,"Attempt":0,"Launch Time":1427397515002,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515082,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":78,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":590793,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":32,"Index":32,"Attempt":0,"Launch Time":1427397515107,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":30,"Index":30,"Attempt":0,"Launch Time":1427397515074,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515108,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":31,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":712959,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":33,"Index":33,"Attempt":0,"Launch Time":1427397515111,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":26,"Index":26,"Attempt":0,"Launch Time":1427397515060,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515112,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":49,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":1133685,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":34,"Index":34,"Attempt":0,"Launch Time":1427397515113,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":27,"Index":27,"Attempt":0,"Launch Time":1427397515062,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515113,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":49,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":647955,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":35,"Index":35,"Attempt":0,"Launch Time":1427397515127,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":25,"Index":25,"Attempt":0,"Launch Time":1427397515059,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515127,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":65,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":487315,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":36,"Index":36,"Attempt":0,"Launch Time":1427397515128,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":31,"Index":31,"Attempt":0,"Launch Time":1427397515082,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515129,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":43,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":1173343,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":37,"Index":37,"Attempt":0,"Launch Time":1427397515134,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":32,"Index":32,"Attempt":0,"Launch Time":1427397515107,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515134,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":25,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":542166,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":38,"Index":38,"Attempt":0,"Launch Time":1427397515135,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":39,"Index":39,"Attempt":0,"Launch Time":1427397515135,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":24,"Index":24,"Attempt":0,"Launch Time":1427397515055,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515135,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":69,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":17403787,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":28,"Index":28,"Attempt":0,"Launch Time":1427397515069,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515135,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":63,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":870321,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":40,"Index":40,"Attempt":0,"Launch Time":1427397515145,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":29,"Index":29,"Attempt":0,"Launch Time":1427397515074,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515146,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":69,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":859166,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":41,"Index":41,"Attempt":0,"Launch Time":1427397515168,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":39,"Index":39,"Attempt":0,"Launch Time":1427397515135,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515168,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":30,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":711567,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":42,"Index":42,"Attempt":0,"Launch Time":1427397515184,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":34,"Index":34,"Attempt":0,"Launch Time":1427397515113,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515184,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":59,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":506890,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":43,"Index":43,"Attempt":0,"Launch Time":1427397515188,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":36,"Index":36,"Attempt":0,"Launch Time":1427397515128,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515188,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":58,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1552336,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":44,"Index":44,"Attempt":0,"Launch Time":1427397515189,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":37,"Index":37,"Attempt":0,"Launch Time":1427397515134,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515189,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":52,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":502961,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":45,"Index":45,"Attempt":0,"Launch Time":1427397515198,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":38,"Index":38,"Attempt":0,"Launch Time":1427397515135,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515198,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":59,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":713958,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":46,"Index":46,"Attempt":0,"Launch Time":1427397515204,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":41,"Index":41,"Attempt":0,"Launch Time":1427397515168,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515205,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":33,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":775901,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":47,"Index":47,"Attempt":0,"Launch Time":1427397515207,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":35,"Index":35,"Attempt":0,"Launch Time":1427397515127,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515208,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":77,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":852701,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":48,"Index":48,"Attempt":0,"Launch Time":1427397515211,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":33,"Index":33,"Attempt":0,"Launch Time":1427397515111,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515211,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":98,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":4651370,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":49,"Index":49,"Attempt":0,"Launch Time":1427397515217,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":40,"Index":40,"Attempt":0,"Launch Time":1427397515145,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515218,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":69,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1798770,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":50,"Index":50,"Attempt":0,"Launch Time":1427397515242,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":46,"Index":46,"Attempt":0,"Launch Time":1427397515204,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515243,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":35,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":411722,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":51,"Index":51,"Attempt":0,"Launch Time":1427397515251,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":45,"Index":45,"Attempt":0,"Launch Time":1427397515198,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515251,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":51,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":888369,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":52,"Index":52,"Attempt":0,"Launch Time":1427397515252,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":48,"Index":48,"Attempt":0,"Launch Time":1427397515211,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515252,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":39,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":525960,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":53,"Index":53,"Attempt":0,"Launch Time":1427397515253,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":42,"Index":42,"Attempt":0,"Launch Time":1427397515184,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515253,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":66,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1901209,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":54,"Index":54,"Attempt":0,"Launch Time":1427397515255,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":47,"Index":47,"Attempt":0,"Launch Time":1427397515207,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515255,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":46,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":812239,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":55,"Index":55,"Attempt":0,"Launch Time":1427397515264,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":44,"Index":44,"Attempt":0,"Launch Time":1427397515189,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515264,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":52,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":373618,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":56,"Index":56,"Attempt":0,"Launch Time":1427397515265,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":49,"Index":49,"Attempt":0,"Launch Time":1427397515217,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515265,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":44,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1742885,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":57,"Index":57,"Attempt":0,"Launch Time":1427397515276,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":50,"Index":50,"Attempt":0,"Launch Time":1427397515242,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515276,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":31,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":486771,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":58,"Index":58,"Attempt":0,"Launch Time":1427397515281,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":43,"Index":43,"Attempt":0,"Launch Time":1427397515188,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515281,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":90,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1036134,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":59,"Index":59,"Attempt":0,"Launch Time":1427397515303,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":52,"Index":52,"Attempt":0,"Launch Time":1427397515252,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515303,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":49,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":733581,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":60,"Index":60,"Attempt":0,"Launch Time":1427397515304,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":51,"Index":51,"Attempt":0,"Launch Time":1427397515251,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515304,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":38,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":857448,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":61,"Index":61,"Attempt":0,"Launch Time":1427397515306,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":54,"Index":54,"Attempt":0,"Launch Time":1427397515255,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515306,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":50,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":822807,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":62,"Index":62,"Attempt":0,"Launch Time":1427397515308,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":56,"Index":56,"Attempt":0,"Launch Time":1427397515265,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515308,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":38,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":597860,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":63,"Index":63,"Attempt":0,"Launch Time":1427397515339,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":57,"Index":57,"Attempt":0,"Launch Time":1427397515276,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515340,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":61,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1682586,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":64,"Index":64,"Attempt":0,"Launch Time":1427397515343,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":53,"Index":53,"Attempt":0,"Launch Time":1427397515253,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515344,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":88,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":1918631,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":65,"Index":65,"Attempt":0,"Launch Time":1427397515347,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":61,"Index":61,"Attempt":0,"Launch Time":1427397515306,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515347,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":39,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":570824,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":66,"Index":66,"Attempt":0,"Launch Time":1427397515362,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":55,"Index":55,"Attempt":0,"Launch Time":1427397515264,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515363,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":96,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":467801,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":67,"Index":67,"Attempt":0,"Launch Time":1427397515374,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":58,"Index":58,"Attempt":0,"Launch Time":1427397515281,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515374,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":91,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":730395,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":68,"Index":68,"Attempt":0,"Launch Time":1427397515375,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":59,"Index":59,"Attempt":0,"Launch Time":1427397515303,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515375,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":69,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9610,"Shuffle Write Time":885679,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":69,"Index":69,"Attempt":0,"Launch Time":1427397515394,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":64,"Index":64,"Attempt":0,"Launch Time":1427397515343,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515394,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":43,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":701822,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":70,"Index":70,"Attempt":0,"Launch Time":1427397515395,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":66,"Index":66,"Attempt":0,"Launch Time":1427397515362,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515395,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":30,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":724051,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":71,"Index":71,"Attempt":0,"Launch Time":1427397515405,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":60,"Index":60,"Attempt":0,"Launch Time":1427397515304,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515405,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":99,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1235948,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":72,"Index":72,"Attempt":0,"Launch Time":1427397515410,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":62,"Index":62,"Attempt":0,"Launch Time":1427397515308,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515410,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":100,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1249286,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":73,"Index":73,"Attempt":0,"Launch Time":1427397515413,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":63,"Index":63,"Attempt":0,"Launch Time":1427397515339,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515413,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":71,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1628301,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":74,"Index":74,"Attempt":0,"Launch Time":1427397515414,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":68,"Index":68,"Attempt":0,"Launch Time":1427397515375,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515415,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":37,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1319579,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":75,"Index":75,"Attempt":0,"Launch Time":1427397515435,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":65,"Index":65,"Attempt":0,"Launch Time":1427397515347,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515436,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":84,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":940133,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":76,"Index":76,"Attempt":0,"Launch Time":1427397515439,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":70,"Index":70,"Attempt":0,"Launch Time":1427397515395,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515440,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":43,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1161930,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":77,"Index":77,"Attempt":0,"Launch Time":1427397515445,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":67,"Index":67,"Attempt":0,"Launch Time":1427397515374,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515445,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":68,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9614,"Shuffle Write Time":4875074,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":78,"Index":78,"Attempt":0,"Launch Time":1427397515452,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":71,"Index":71,"Attempt":0,"Launch Time":1427397515405,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515453,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":45,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1422810,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":79,"Index":79,"Attempt":0,"Launch Time":1427397515454,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":72,"Index":72,"Attempt":0,"Launch Time":1427397515410,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515454,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":42,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1311510,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":80,"Index":80,"Attempt":0,"Launch Time":1427397515459,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":73,"Index":73,"Attempt":0,"Launch Time":1427397515413,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515459,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":44,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1180217,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":81,"Index":81,"Attempt":0,"Launch Time":1427397515462,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":69,"Index":69,"Attempt":0,"Launch Time":1427397515394,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515463,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":67,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":1136518,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":82,"Index":82,"Attempt":0,"Launch Time":1427397515486,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":74,"Index":74,"Attempt":0,"Launch Time":1427397515414,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515487,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":71,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":774329,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":83,"Index":83,"Attempt":0,"Launch Time":1427397515491,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":75,"Index":75,"Attempt":0,"Launch Time":1427397515435,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515491,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":54,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1443592,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":84,"Index":84,"Attempt":0,"Launch Time":1427397515503,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":77,"Index":77,"Attempt":0,"Launch Time":1427397515445,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515503,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":56,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":21293788,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":85,"Index":85,"Attempt":0,"Launch Time":1427397515510,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":76,"Index":76,"Attempt":0,"Launch Time":1427397515439,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515510,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":68,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1488019,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":86,"Index":86,"Attempt":0,"Launch Time":1427397515512,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":81,"Index":81,"Attempt":0,"Launch Time":1427397515462,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515512,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":47,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1596086,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":87,"Index":87,"Attempt":0,"Launch Time":1427397515521,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":78,"Index":78,"Attempt":0,"Launch Time":1427397515452,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515521,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":66,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1645424,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":88,"Index":88,"Attempt":0,"Launch Time":1427397515522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":80,"Index":80,"Attempt":0,"Launch Time":1427397515459,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515522,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":61,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1552735,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":89,"Index":89,"Attempt":0,"Launch Time":1427397515526,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":79,"Index":79,"Attempt":0,"Launch Time":1427397515454,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515527,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":71,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":3022416,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":90,"Index":90,"Attempt":0,"Launch Time":1427397515535,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":82,"Index":82,"Attempt":0,"Launch Time":1427397515486,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515535,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":47,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":878266,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":91,"Index":91,"Attempt":0,"Launch Time":1427397515550,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":85,"Index":85,"Attempt":0,"Launch Time":1427397515510,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515550,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":38,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":745599,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":92,"Index":92,"Attempt":0,"Launch Time":1427397515552,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":83,"Index":83,"Attempt":0,"Launch Time":1427397515491,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515553,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":60,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1183846,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":93,"Index":93,"Attempt":0,"Launch Time":1427397515553,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":84,"Index":84,"Attempt":0,"Launch Time":1427397515503,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515553,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":49,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1082699,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":94,"Index":94,"Attempt":0,"Launch Time":1427397515558,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":86,"Index":86,"Attempt":0,"Launch Time":1427397515512,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515559,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":45,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1656735,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":95,"Index":95,"Attempt":0,"Launch Time":1427397515571,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":87,"Index":87,"Attempt":0,"Launch Time":1427397515521,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515571,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":48,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":705517,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":96,"Index":96,"Attempt":0,"Launch Time":1427397515577,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":89,"Index":89,"Attempt":0,"Launch Time":1427397515526,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515577,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":48,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":588365,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":97,"Index":97,"Attempt":0,"Launch Time":1427397515580,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":88,"Index":88,"Attempt":0,"Launch Time":1427397515522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515581,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":47,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1437387,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":98,"Index":98,"Attempt":0,"Launch Time":1427397515585,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":90,"Index":90,"Attempt":0,"Launch Time":1427397515535,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515586,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":49,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":2067069,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":99,"Index":99,"Attempt":0,"Launch Time":1427397515595,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":91,"Index":91,"Attempt":0,"Launch Time":1427397515550,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515596,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":44,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":2560008,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":100,"Index":100,"Attempt":0,"Launch Time":1427397515600,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":93,"Index":93,"Attempt":0,"Launch Time":1427397515553,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515600,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":44,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1545777,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":101,"Index":101,"Attempt":0,"Launch Time":1427397515608,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":94,"Index":94,"Attempt":0,"Launch Time":1427397515558,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515608,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":47,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":643678,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":102,"Index":102,"Attempt":0,"Launch Time":1427397515628,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":95,"Index":95,"Attempt":0,"Launch Time":1427397515571,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515629,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":56,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1091283,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":103,"Index":103,"Attempt":0,"Launch Time":1427397515631,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":97,"Index":97,"Attempt":0,"Launch Time":1427397515580,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515631,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":49,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1737597,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":104,"Index":104,"Attempt":0,"Launch Time":1427397515637,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":92,"Index":92,"Attempt":0,"Launch Time":1427397515552,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515637,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":83,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1004616,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":105,"Index":105,"Attempt":0,"Launch Time":1427397515639,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":98,"Index":98,"Attempt":0,"Launch Time":1427397515585,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515640,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":52,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":5664832,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":106,"Index":106,"Attempt":0,"Launch Time":1427397515649,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":96,"Index":96,"Attempt":0,"Launch Time":1427397515577,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515649,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":70,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":835020,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":107,"Index":107,"Attempt":0,"Launch Time":1427397515655,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":101,"Index":101,"Attempt":0,"Launch Time":1427397515608,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515656,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":46,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":2692928,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":108,"Index":108,"Attempt":0,"Launch Time":1427397515665,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":102,"Index":102,"Attempt":0,"Launch Time":1427397515628,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515665,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":35,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":806822,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":109,"Index":109,"Attempt":0,"Launch Time":1427397515667,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":104,"Index":104,"Attempt":0,"Launch Time":1427397515637,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515668,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":29,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":629448,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":110,"Index":110,"Attempt":0,"Launch Time":1427397515680,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":105,"Index":105,"Attempt":0,"Launch Time":1427397515639,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515680,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":38,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1592879,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":111,"Index":111,"Attempt":0,"Launch Time":1427397515680,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":100,"Index":100,"Attempt":0,"Launch Time":1427397515600,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515680,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":79,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":5717688,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":112,"Index":112,"Attempt":0,"Launch Time":1427397515683,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":103,"Index":103,"Attempt":0,"Launch Time":1427397515631,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515684,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":51,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":2110758,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":113,"Index":113,"Attempt":0,"Launch Time":1427397515686,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":99,"Index":99,"Attempt":0,"Launch Time":1427397515595,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515686,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":87,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":2827506,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":114,"Index":114,"Attempt":0,"Launch Time":1427397515689,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":106,"Index":106,"Attempt":0,"Launch Time":1427397515649,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515689,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":38,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":993259,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":115,"Index":115,"Attempt":0,"Launch Time":1427397515703,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":107,"Index":107,"Attempt":0,"Launch Time":1427397515655,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515703,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":46,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":901718,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":116,"Index":116,"Attempt":0,"Launch Time":1427397515712,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":111,"Index":111,"Attempt":0,"Launch Time":1427397515680,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515712,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":29,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":665220,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":117,"Index":117,"Attempt":0,"Launch Time":1427397515719,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":109,"Index":109,"Attempt":0,"Launch Time":1427397515667,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515720,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":51,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":2188354,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":118,"Index":118,"Attempt":0,"Launch Time":1427397515720,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":108,"Index":108,"Attempt":0,"Launch Time":1427397515665,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515721,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":53,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1868304,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":119,"Index":119,"Attempt":0,"Launch Time":1427397515723,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":113,"Index":113,"Attempt":0,"Launch Time":1427397515686,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515723,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":35,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1575183,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":120,"Index":120,"Attempt":0,"Launch Time":1427397515736,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":114,"Index":114,"Attempt":0,"Launch Time":1427397515689,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515737,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":44,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":824387,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":121,"Index":121,"Attempt":0,"Launch Time":1427397515742,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":112,"Index":112,"Attempt":0,"Launch Time":1427397515683,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515743,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":57,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":919762,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":122,"Index":122,"Attempt":0,"Launch Time":1427397515756,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":117,"Index":117,"Attempt":0,"Launch Time":1427397515719,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515756,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":34,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1330006,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":123,"Index":123,"Attempt":0,"Launch Time":1427397515762,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":115,"Index":115,"Attempt":0,"Launch Time":1427397515703,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515762,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":57,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1796871,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":124,"Index":124,"Attempt":0,"Launch Time":1427397515766,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":110,"Index":110,"Attempt":0,"Launch Time":1427397515680,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515766,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":84,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1387335,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":125,"Index":125,"Attempt":0,"Launch Time":1427397515778,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":116,"Index":116,"Attempt":0,"Launch Time":1427397515712,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515778,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":65,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1125262,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":126,"Index":126,"Attempt":0,"Launch Time":1427397515785,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":120,"Index":120,"Attempt":0,"Launch Time":1427397515736,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515786,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":46,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":632892,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":127,"Index":127,"Attempt":0,"Launch Time":1427397515795,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":118,"Index":118,"Attempt":0,"Launch Time":1427397515720,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515796,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":71,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1530208,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":128,"Index":128,"Attempt":0,"Launch Time":1427397515796,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":119,"Index":119,"Attempt":0,"Launch Time":1427397515723,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515796,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":72,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":2231579,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":129,"Index":129,"Attempt":0,"Launch Time":1427397515802,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":123,"Index":123,"Attempt":0,"Launch Time":1427397515762,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515802,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":39,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1211775,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":130,"Index":130,"Attempt":0,"Launch Time":1427397515804,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":121,"Index":121,"Attempt":0,"Launch Time":1427397515742,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515804,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":59,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1650668,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":131,"Index":131,"Attempt":0,"Launch Time":1427397515804,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":124,"Index":124,"Attempt":0,"Launch Time":1427397515766,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515805,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":37,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":654729,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":132,"Index":132,"Attempt":0,"Launch Time":1427397515819,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":122,"Index":122,"Attempt":0,"Launch Time":1427397515756,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515819,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":61,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":626171,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":133,"Index":133,"Attempt":0,"Launch Time":1427397515828,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":125,"Index":125,"Attempt":0,"Launch Time":1427397515778,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515828,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":48,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":788072,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":134,"Index":134,"Attempt":0,"Launch Time":1427397515833,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":127,"Index":127,"Attempt":0,"Launch Time":1427397515795,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515833,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":34,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1102978,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":135,"Index":135,"Attempt":0,"Launch Time":1427397515836,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":128,"Index":128,"Attempt":0,"Launch Time":1427397515796,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515836,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":34,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":742957,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":136,"Index":136,"Attempt":0,"Launch Time":1427397515843,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":126,"Index":126,"Attempt":0,"Launch Time":1427397515785,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515843,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":54,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":1350040,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":137,"Index":137,"Attempt":0,"Launch Time":1427397515860,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":132,"Index":132,"Attempt":0,"Launch Time":1427397515819,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515860,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":39,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":565467,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":138,"Index":138,"Attempt":0,"Launch Time":1427397515862,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":131,"Index":131,"Attempt":0,"Launch Time":1427397515804,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515862,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":57,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":2514923,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":139,"Index":139,"Attempt":0,"Launch Time":1427397515871,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":133,"Index":133,"Attempt":0,"Launch Time":1427397515828,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515872,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":42,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":762398,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":140,"Index":140,"Attempt":0,"Launch Time":1427397515873,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":141,"Index":141,"Attempt":0,"Launch Time":1427397515873,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":130,"Index":130,"Attempt":0,"Launch Time":1427397515804,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515873,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":67,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1051478,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":136,"Index":136,"Attempt":0,"Launch Time":1427397515843,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515873,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":28,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1970360,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":142,"Index":142,"Attempt":0,"Launch Time":1427397515879,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":129,"Index":129,"Attempt":0,"Launch Time":1427397515802,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515879,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":75,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1143246,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":143,"Index":143,"Attempt":0,"Launch Time":1427397515899,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":134,"Index":134,"Attempt":0,"Launch Time":1427397515833,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515900,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":65,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":11485852,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":144,"Index":144,"Attempt":0,"Launch Time":1427397515925,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":138,"Index":138,"Attempt":0,"Launch Time":1427397515862,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515925,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":54,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":818784,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":145,"Index":145,"Attempt":0,"Launch Time":1427397515925,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":135,"Index":135,"Attempt":0,"Launch Time":1427397515836,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515926,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":87,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1145231,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":146,"Index":146,"Attempt":0,"Launch Time":1427397515927,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":142,"Index":142,"Attempt":0,"Launch Time":1427397515879,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515927,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":47,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":456410,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":147,"Index":147,"Attempt":0,"Launch Time":1427397515934,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":141,"Index":141,"Attempt":0,"Launch Time":1427397515873,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515934,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":59,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":5319888,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":148,"Index":148,"Attempt":0,"Launch Time":1427397515939,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":139,"Index":139,"Attempt":0,"Launch Time":1427397515871,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515939,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":66,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1180285,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":149,"Index":149,"Attempt":0,"Launch Time":1427397515940,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":137,"Index":137,"Attempt":0,"Launch Time":1427397515860,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515941,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":79,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1130140,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":150,"Index":150,"Attempt":0,"Launch Time":1427397515955,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":146,"Index":146,"Attempt":0,"Launch Time":1427397515927,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515955,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":26,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1146878,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":151,"Index":151,"Attempt":0,"Launch Time":1427397515968,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":140,"Index":140,"Attempt":0,"Launch Time":1427397515873,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515968,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":93,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9606,"Shuffle Write Time":888469,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":152,"Index":152,"Attempt":0,"Launch Time":1427397515969,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":143,"Index":143,"Attempt":0,"Launch Time":1427397515899,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515969,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":67,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":554532,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":153,"Index":153,"Attempt":0,"Launch Time":1427397515979,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":150,"Index":150,"Attempt":0,"Launch Time":1427397515955,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515979,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":22,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":880380,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":154,"Index":154,"Attempt":0,"Launch Time":1427397515983,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":145,"Index":145,"Attempt":0,"Launch Time":1427397515925,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515983,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":56,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":1803512,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":155,"Index":155,"Attempt":0,"Launch Time":1427397515997,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":147,"Index":147,"Attempt":0,"Launch Time":1427397515934,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515997,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":60,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":803764,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":156,"Index":156,"Attempt":0,"Launch Time":1427397516001,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":148,"Index":148,"Attempt":0,"Launch Time":1427397515939,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516002,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":61,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1352276,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":157,"Index":157,"Attempt":0,"Launch Time":1427397516007,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":153,"Index":153,"Attempt":0,"Launch Time":1427397515979,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516008,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":27,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1319166,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":158,"Index":158,"Attempt":0,"Launch Time":1427397516025,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":144,"Index":144,"Attempt":0,"Launch Time":1427397515925,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516025,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":97,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":2039278,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":159,"Index":159,"Attempt":0,"Launch Time":1427397516032,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":157,"Index":157,"Attempt":0,"Launch Time":1427397516007,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516032,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":24,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":982378,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":160,"Index":160,"Attempt":0,"Launch Time":1427397516034,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":149,"Index":149,"Attempt":0,"Launch Time":1427397515940,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516035,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":92,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":2072494,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":161,"Index":161,"Attempt":0,"Launch Time":1427397516046,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":152,"Index":152,"Attempt":0,"Launch Time":1427397515969,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516046,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":75,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":2115526,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":162,"Index":162,"Attempt":0,"Launch Time":1427397516047,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":151,"Index":151,"Attempt":0,"Launch Time":1427397515968,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516047,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":76,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":802352,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":163,"Index":163,"Attempt":0,"Launch Time":1427397516059,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":154,"Index":154,"Attempt":0,"Launch Time":1427397515983,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516059,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":73,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":3726182,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":164,"Index":164,"Attempt":0,"Launch Time":1427397516102,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":163,"Index":163,"Attempt":0,"Launch Time":1427397516059,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516102,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":41,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":3131755,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":165,"Index":165,"Attempt":0,"Launch Time":1427397516104,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":155,"Index":155,"Attempt":0,"Launch Time":1427397515997,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516104,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":105,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":3474912,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":166,"Index":166,"Attempt":0,"Launch Time":1427397516107,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":159,"Index":159,"Attempt":0,"Launch Time":1427397516032,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516107,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":73,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":31801636,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":167,"Index":167,"Attempt":0,"Launch Time":1427397516117,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":160,"Index":160,"Attempt":0,"Launch Time":1427397516034,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516117,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":81,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":8899144,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":168,"Index":168,"Attempt":0,"Launch Time":1427397516119,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":158,"Index":158,"Attempt":0,"Launch Time":1427397516025,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516119,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":90,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":8724838,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":169,"Index":169,"Attempt":0,"Launch Time":1427397516124,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":156,"Index":156,"Attempt":0,"Launch Time":1427397516001,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516125,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":121,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":23115082,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":170,"Index":170,"Attempt":0,"Launch Time":1427397516135,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":162,"Index":162,"Attempt":0,"Launch Time":1427397516047,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516135,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":86,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":17488871,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":171,"Index":171,"Attempt":0,"Launch Time":1427397516137,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":161,"Index":161,"Attempt":0,"Launch Time":1427397516046,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516137,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":88,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1397930,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":172,"Index":172,"Attempt":0,"Launch Time":1427397516157,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":169,"Index":169,"Attempt":0,"Launch Time":1427397516124,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516157,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":31,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1286286,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":173,"Index":173,"Attempt":0,"Launch Time":1427397516163,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":168,"Index":168,"Attempt":0,"Launch Time":1427397516119,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516164,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":43,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1164757,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":174,"Index":174,"Attempt":0,"Launch Time":1427397516166,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":167,"Index":167,"Attempt":0,"Launch Time":1427397516117,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516166,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":48,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1257435,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":175,"Index":175,"Attempt":0,"Launch Time":1427397516172,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":165,"Index":165,"Attempt":0,"Launch Time":1427397516104,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516172,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":66,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":857534,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":176,"Index":176,"Attempt":0,"Launch Time":1427397516175,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":170,"Index":170,"Attempt":0,"Launch Time":1427397516135,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516176,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":38,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1930750,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":177,"Index":177,"Attempt":0,"Launch Time":1427397516183,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":166,"Index":166,"Attempt":0,"Launch Time":1427397516107,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516183,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":74,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":918242,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":178,"Index":178,"Attempt":0,"Launch Time":1427397516199,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":171,"Index":171,"Attempt":0,"Launch Time":1427397516137,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516199,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":61,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":601779,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":179,"Index":179,"Attempt":0,"Launch Time":1427397516200,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":164,"Index":164,"Attempt":0,"Launch Time":1427397516102,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516200,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":97,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1163035,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":180,"Index":180,"Attempt":0,"Launch Time":1427397516212,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":172,"Index":172,"Attempt":0,"Launch Time":1427397516157,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516212,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":37,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":3053113,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":181,"Index":181,"Attempt":0,"Launch Time":1427397516221,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":182,"Index":182,"Attempt":0,"Launch Time":1427397516222,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":177,"Index":177,"Attempt":0,"Launch Time":1427397516183,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516222,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":37,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":729369,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":175,"Index":175,"Attempt":0,"Launch Time":1427397516172,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516222,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":44,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":862204,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":183,"Index":183,"Attempt":0,"Launch Time":1427397516236,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":173,"Index":173,"Attempt":0,"Launch Time":1427397516163,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516237,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":72,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":2386820,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":184,"Index":184,"Attempt":0,"Launch Time":1427397516240,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":174,"Index":174,"Attempt":0,"Launch Time":1427397516166,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516241,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":73,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":807851,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":185,"Index":185,"Attempt":0,"Launch Time":1427397516247,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":178,"Index":178,"Attempt":0,"Launch Time":1427397516199,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516262,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":45,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":758218,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":186,"Index":186,"Attempt":0,"Launch Time":1427397516262,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":176,"Index":176,"Attempt":0,"Launch Time":1427397516175,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516263,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":70,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":691124,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":187,"Index":187,"Attempt":0,"Launch Time":1427397516276,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":179,"Index":179,"Attempt":0,"Launch Time":1427397516200,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516277,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":76,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1328811,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":188,"Index":188,"Attempt":0,"Launch Time":1427397516283,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":180,"Index":180,"Attempt":0,"Launch Time":1427397516212,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516284,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":70,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":599086,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":189,"Index":189,"Attempt":0,"Launch Time":1427397516287,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":181,"Index":181,"Attempt":0,"Launch Time":1427397516221,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516287,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":63,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1475983,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":190,"Index":190,"Attempt":0,"Launch Time":1427397516299,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":183,"Index":183,"Attempt":0,"Launch Time":1427397516236,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516299,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":61,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":2833401,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":191,"Index":191,"Attempt":0,"Launch Time":1427397516300,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":184,"Index":184,"Attempt":0,"Launch Time":1427397516240,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516300,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":58,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":739937,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":192,"Index":192,"Attempt":0,"Launch Time":1427397516304,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":182,"Index":182,"Attempt":0,"Launch Time":1427397516222,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516305,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":81,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1404765,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":193,"Index":193,"Attempt":0,"Launch Time":1427397516313,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":185,"Index":185,"Attempt":0,"Launch Time":1427397516247,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516314,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":49,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":561301,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":194,"Index":194,"Attempt":0,"Launch Time":1427397516319,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":186,"Index":186,"Attempt":0,"Launch Time":1427397516262,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516320,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":55,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1564335,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":195,"Index":195,"Attempt":0,"Launch Time":1427397516327,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":187,"Index":187,"Attempt":0,"Launch Time":1427397516276,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516327,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":48,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":989979,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":196,"Index":196,"Attempt":0,"Launch Time":1427397516334,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":188,"Index":188,"Attempt":0,"Launch Time":1427397516283,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516334,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":49,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1382934,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":197,"Index":197,"Attempt":0,"Launch Time":1427397516343,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":192,"Index":192,"Attempt":0,"Launch Time":1427397516304,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516344,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":38,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":711388,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":198,"Index":198,"Attempt":0,"Launch Time":1427397516344,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":190,"Index":190,"Attempt":0,"Launch Time":1427397516299,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516344,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":43,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":495051,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":199,"Index":199,"Attempt":0,"Launch Time":1427397516348,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":189,"Index":189,"Attempt":0,"Launch Time":1427397516287,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516349,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":60,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":799667,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":200,"Index":200,"Attempt":0,"Launch Time":1427397516360,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":191,"Index":191,"Attempt":0,"Launch Time":1427397516300,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516361,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":60,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":663667,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":201,"Index":201,"Attempt":0,"Launch Time":1427397516370,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":193,"Index":193,"Attempt":0,"Launch Time":1427397516313,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516370,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":54,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":895416,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":202,"Index":202,"Attempt":0,"Launch Time":1427397516377,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":194,"Index":194,"Attempt":0,"Launch Time":1427397516319,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516377,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":55,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":995102,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":203,"Index":203,"Attempt":0,"Launch Time":1427397516377,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":199,"Index":199,"Attempt":0,"Launch Time":1427397516348,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516377,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":27,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1007155,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":204,"Index":204,"Attempt":0,"Launch Time":1427397516384,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":195,"Index":195,"Attempt":0,"Launch Time":1427397516327,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516384,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":56,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":2006866,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":205,"Index":205,"Attempt":0,"Launch Time":1427397516399,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":196,"Index":196,"Attempt":0,"Launch Time":1427397516334,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516399,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":63,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":994986,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":206,"Index":206,"Attempt":0,"Launch Time":1427397516401,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":202,"Index":202,"Attempt":0,"Launch Time":1427397516377,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516401,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":22,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":777981,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":207,"Index":207,"Attempt":0,"Launch Time":1427397516422,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":197,"Index":197,"Attempt":0,"Launch Time":1427397516343,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516423,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":77,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":734290,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":208,"Index":208,"Attempt":0,"Launch Time":1427397516427,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":200,"Index":200,"Attempt":0,"Launch Time":1427397516360,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516427,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":66,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":890678,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":209,"Index":209,"Attempt":0,"Launch Time":1427397516428,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":198,"Index":198,"Attempt":0,"Launch Time":1427397516344,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516429,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":82,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":2042848,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":210,"Index":210,"Attempt":0,"Launch Time":1427397516436,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":206,"Index":206,"Attempt":0,"Launch Time":1427397516401,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516436,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":35,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":915341,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":211,"Index":211,"Attempt":0,"Launch Time":1427397516439,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":205,"Index":205,"Attempt":0,"Launch Time":1427397516399,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516439,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":38,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":883846,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":212,"Index":212,"Attempt":0,"Launch Time":1427397516450,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":204,"Index":204,"Attempt":0,"Launch Time":1427397516384,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516450,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":64,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":690408,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":213,"Index":213,"Attempt":0,"Launch Time":1427397516450,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":201,"Index":201,"Attempt":0,"Launch Time":1427397516370,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516450,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":78,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":2085369,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":214,"Index":214,"Attempt":0,"Launch Time":1427397516454,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":203,"Index":203,"Attempt":0,"Launch Time":1427397516377,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516454,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":75,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1600531,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":215,"Index":215,"Attempt":0,"Launch Time":1427397516467,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":210,"Index":210,"Attempt":0,"Launch Time":1427397516436,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516468,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":30,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":707402,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":216,"Index":216,"Attempt":0,"Launch Time":1427397516478,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":213,"Index":213,"Attempt":0,"Launch Time":1427397516450,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516478,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":25,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":5862185,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":217,"Index":217,"Attempt":0,"Launch Time":1427397516478,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":212,"Index":212,"Attempt":0,"Launch Time":1427397516450,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516479,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":26,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":5903505,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":218,"Index":218,"Attempt":0,"Launch Time":1427397516479,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":214,"Index":214,"Attempt":0,"Launch Time":1427397516454,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516479,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":24,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1108244,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":219,"Index":219,"Attempt":0,"Launch Time":1427397516482,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":215,"Index":215,"Attempt":0,"Launch Time":1427397516467,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516482,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1424767,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":220,"Index":220,"Attempt":0,"Launch Time":1427397516497,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":221,"Index":221,"Attempt":0,"Launch Time":1427397516498,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":216,"Index":216,"Attempt":0,"Launch Time":1427397516478,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516498,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":2152877,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":219,"Index":219,"Attempt":0,"Launch Time":1427397516482,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516498,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1170174,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":222,"Index":222,"Attempt":0,"Launch Time":1427397516502,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":218,"Index":218,"Attempt":0,"Launch Time":1427397516479,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516503,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":22,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":2103594,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":223,"Index":223,"Attempt":0,"Launch Time":1427397516505,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":217,"Index":217,"Attempt":0,"Launch Time":1427397516478,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516506,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":25,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":2305044,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":224,"Index":224,"Attempt":0,"Launch Time":1427397516516,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":221,"Index":221,"Attempt":0,"Launch Time":1427397516498,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516516,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1233130,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":225,"Index":225,"Attempt":0,"Launch Time":1427397516519,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":222,"Index":222,"Attempt":0,"Launch Time":1427397516502,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516519,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1335051,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":226,"Index":226,"Attempt":0,"Launch Time":1427397516522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":220,"Index":220,"Attempt":0,"Launch Time":1427397516497,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516522,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":24,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":3418527,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":227,"Index":227,"Attempt":0,"Launch Time":1427397516522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":223,"Index":223,"Attempt":0,"Launch Time":1427397516505,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516523,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":914172,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":228,"Index":228,"Attempt":0,"Launch Time":1427397516529,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":208,"Index":208,"Attempt":0,"Launch Time":1427397516427,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516529,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":100,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1002051,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":229,"Index":229,"Attempt":0,"Launch Time":1427397516531,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":207,"Index":207,"Attempt":0,"Launch Time":1427397516422,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516531,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":106,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":3672588,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":230,"Index":230,"Attempt":0,"Launch Time":1427397516558,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":209,"Index":209,"Attempt":0,"Launch Time":1427397516428,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516558,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":128,"Result Size":930,"JVM GC Time":17,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":2462354,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":231,"Index":231,"Attempt":0,"Launch Time":1427397516559,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":211,"Index":211,"Attempt":0,"Launch Time":1427397516439,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516559,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":118,"Result Size":930,"JVM GC Time":17,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":3121161,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":232,"Index":232,"Attempt":0,"Launch Time":1427397516565,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":228,"Index":228,"Attempt":0,"Launch Time":1427397516529,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516565,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":33,"Result Size":930,"JVM GC Time":17,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1682298,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":233,"Index":233,"Attempt":0,"Launch Time":1427397516567,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":224,"Index":224,"Attempt":0,"Launch Time":1427397516516,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516567,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":49,"Result Size":930,"JVM GC Time":17,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":33955650,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":234,"Index":234,"Attempt":0,"Launch Time":1427397516571,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":229,"Index":229,"Attempt":0,"Launch Time":1427397516531,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516571,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":36,"Result Size":930,"JVM GC Time":17,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":1487266,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":235,"Index":235,"Attempt":0,"Launch Time":1427397516573,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":230,"Index":230,"Attempt":0,"Launch Time":1427397516558,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516574,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1457248,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":236,"Index":236,"Attempt":0,"Launch Time":1427397516575,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":231,"Index":231,"Attempt":0,"Launch Time":1427397516559,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516575,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":929760,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":237,"Index":237,"Attempt":0,"Launch Time":1427397516580,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":232,"Index":232,"Attempt":0,"Launch Time":1427397516565,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516580,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":976217,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":238,"Index":238,"Attempt":0,"Launch Time":1427397516587,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":234,"Index":234,"Attempt":0,"Launch Time":1427397516571,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516588,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1735408,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":239,"Index":239,"Attempt":0,"Launch Time":1427397516589,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":235,"Index":235,"Attempt":0,"Launch Time":1427397516573,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516589,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1858962,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":240,"Index":240,"Attempt":0,"Launch Time":1427397516591,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":236,"Index":236,"Attempt":0,"Launch Time":1427397516575,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516592,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1261207,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":241,"Index":241,"Attempt":0,"Launch Time":1427397516597,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":237,"Index":237,"Attempt":0,"Launch Time":1427397516580,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516597,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1060138,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":242,"Index":242,"Attempt":0,"Launch Time":1427397516598,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":233,"Index":233,"Attempt":0,"Launch Time":1427397516567,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516598,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":29,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":2912642,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":243,"Index":243,"Attempt":0,"Launch Time":1427397516602,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":244,"Index":244,"Attempt":0,"Launch Time":1427397516602,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":225,"Index":225,"Attempt":0,"Launch Time":1427397516519,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516602,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":82,"Result Size":930,"JVM GC Time":17,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":40931747,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":238,"Index":238,"Attempt":0,"Launch Time":1427397516587,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516602,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1613089,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":245,"Index":245,"Attempt":0,"Launch Time":1427397516604,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":239,"Index":239,"Attempt":0,"Launch Time":1427397516589,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516604,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1335202,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":246,"Index":246,"Attempt":0,"Launch Time":1427397516607,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":240,"Index":240,"Attempt":0,"Launch Time":1427397516591,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516607,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1712197,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":247,"Index":247,"Attempt":0,"Launch Time":1427397516612,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":242,"Index":242,"Attempt":0,"Launch Time":1427397516598,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516613,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":893554,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":248,"Index":248,"Attempt":0,"Launch Time":1427397516614,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":227,"Index":227,"Attempt":0,"Launch Time":1427397516522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516614,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":60,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":18973299,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":249,"Index":249,"Attempt":0,"Launch Time":1427397516616,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":243,"Index":243,"Attempt":0,"Launch Time":1427397516602,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516617,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1141362,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":250,"Index":250,"Attempt":0,"Launch Time":1427397516617,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":245,"Index":245,"Attempt":0,"Launch Time":1427397516604,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516618,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":866325,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":251,"Index":251,"Attempt":0,"Launch Time":1427397516619,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":241,"Index":241,"Attempt":0,"Launch Time":1427397516597,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516619,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":20,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":2477820,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":252,"Index":252,"Attempt":0,"Launch Time":1427397516621,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":246,"Index":246,"Attempt":0,"Launch Time":1427397516607,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516621,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1132855,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":253,"Index":253,"Attempt":0,"Launch Time":1427397516641,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":247,"Index":247,"Attempt":0,"Launch Time":1427397516612,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516641,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":28,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1005231,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":254,"Index":254,"Attempt":0,"Launch Time":1427397516643,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":250,"Index":250,"Attempt":0,"Launch Time":1427397516617,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516643,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":24,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":689612,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":255,"Index":255,"Attempt":0,"Launch Time":1427397516644,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":226,"Index":226,"Attempt":0,"Launch Time":1427397516522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516645,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":90,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":4229102,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":256,"Index":256,"Attempt":0,"Launch Time":1427397516646,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":249,"Index":249,"Attempt":0,"Launch Time":1427397516616,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516646,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":28,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9613,"Shuffle Write Time":1129918,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":257,"Index":257,"Attempt":0,"Launch Time":1427397516655,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":244,"Index":244,"Attempt":0,"Launch Time":1427397516602,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516655,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":38,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1062798,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":258,"Index":258,"Attempt":0,"Launch Time":1427397516657,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":254,"Index":254,"Attempt":0,"Launch Time":1427397516643,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516657,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1053040,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":259,"Index":259,"Attempt":0,"Launch Time":1427397516662,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":255,"Index":255,"Attempt":0,"Launch Time":1427397516644,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516662,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1372717,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":260,"Index":260,"Attempt":0,"Launch Time":1427397516665,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":248,"Index":248,"Attempt":0,"Launch Time":1427397516614,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516665,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":17,"Executor Run Time":28,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":4140539,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":261,"Index":261,"Attempt":0,"Launch Time":1427397516670,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":258,"Index":258,"Attempt":0,"Launch Time":1427397516657,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516670,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":834919,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":262,"Index":262,"Attempt":0,"Launch Time":1427397516672,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":257,"Index":257,"Attempt":0,"Launch Time":1427397516655,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516672,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9613,"Shuffle Write Time":1313635,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":263,"Index":263,"Attempt":0,"Launch Time":1427397516679,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":259,"Index":259,"Attempt":0,"Launch Time":1427397516662,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516679,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1562479,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":264,"Index":264,"Attempt":0,"Launch Time":1427397516681,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":260,"Index":260,"Attempt":0,"Launch Time":1427397516665,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516681,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1094824,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":265,"Index":265,"Attempt":0,"Launch Time":1427397516681,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":261,"Index":261,"Attempt":0,"Launch Time":1427397516670,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516682,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1053413,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":266,"Index":266,"Attempt":0,"Launch Time":1427397516688,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":262,"Index":262,"Attempt":0,"Launch Time":1427397516672,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516688,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1082688,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":267,"Index":267,"Attempt":0,"Launch Time":1427397516693,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":263,"Index":263,"Attempt":0,"Launch Time":1427397516679,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516694,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":849398,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":268,"Index":268,"Attempt":0,"Launch Time":1427397516695,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":265,"Index":265,"Attempt":0,"Launch Time":1427397516681,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516695,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9613,"Shuffle Write Time":1144571,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":269,"Index":269,"Attempt":0,"Launch Time":1427397516696,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":264,"Index":264,"Attempt":0,"Launch Time":1427397516681,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516696,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1215266,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":270,"Index":270,"Attempt":0,"Launch Time":1427397516702,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":266,"Index":266,"Attempt":0,"Launch Time":1427397516688,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516703,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1082701,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":271,"Index":271,"Attempt":0,"Launch Time":1427397516707,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":267,"Index":267,"Attempt":0,"Launch Time":1427397516693,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516707,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":831860,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":272,"Index":272,"Attempt":0,"Launch Time":1427397516707,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":268,"Index":268,"Attempt":0,"Launch Time":1427397516695,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516708,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":580898,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":273,"Index":273,"Attempt":0,"Launch Time":1427397516709,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":251,"Index":251,"Attempt":0,"Launch Time":1427397516619,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516709,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":88,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":3552935,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":274,"Index":274,"Attempt":0,"Launch Time":1427397516710,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":269,"Index":269,"Attempt":0,"Launch Time":1427397516696,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516711,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1251490,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":275,"Index":275,"Attempt":0,"Launch Time":1427397516730,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":252,"Index":252,"Attempt":0,"Launch Time":1427397516621,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516730,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":107,"Result Size":930,"JVM GC Time":19,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":44499282,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":276,"Index":276,"Attempt":0,"Launch Time":1427397516731,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":270,"Index":270,"Attempt":0,"Launch Time":1427397516702,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516731,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":26,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1226028,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":277,"Index":277,"Attempt":0,"Launch Time":1427397516734,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":271,"Index":271,"Attempt":0,"Launch Time":1427397516707,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516735,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":26,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":909786,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":278,"Index":278,"Attempt":0,"Launch Time":1427397516735,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":274,"Index":274,"Attempt":0,"Launch Time":1427397516710,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516735,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":23,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1240993,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":279,"Index":279,"Attempt":0,"Launch Time":1427397516736,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":272,"Index":272,"Attempt":0,"Launch Time":1427397516707,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516736,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":28,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":626240,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":280,"Index":280,"Attempt":0,"Launch Time":1427397516741,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":273,"Index":273,"Attempt":0,"Launch Time":1427397516709,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516742,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":31,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":2206706,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":281,"Index":281,"Attempt":0,"Launch Time":1427397516744,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":276,"Index":276,"Attempt":0,"Launch Time":1427397516731,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516744,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":877775,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":282,"Index":282,"Attempt":0,"Launch Time":1427397516748,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":278,"Index":278,"Attempt":0,"Launch Time":1427397516735,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516748,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1332523,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":283,"Index":283,"Attempt":0,"Launch Time":1427397516748,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":277,"Index":277,"Attempt":0,"Launch Time":1427397516734,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516748,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":862540,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":284,"Index":284,"Attempt":0,"Launch Time":1427397516750,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":279,"Index":279,"Attempt":0,"Launch Time":1427397516736,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516750,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1521596,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":285,"Index":285,"Attempt":0,"Launch Time":1427397516758,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":281,"Index":281,"Attempt":0,"Launch Time":1427397516744,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516758,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":976300,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":286,"Index":286,"Attempt":0,"Launch Time":1427397516760,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":280,"Index":280,"Attempt":0,"Launch Time":1427397516741,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516760,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":2435944,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":287,"Index":287,"Attempt":0,"Launch Time":1427397516761,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":282,"Index":282,"Attempt":0,"Launch Time":1427397516748,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516762,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":741457,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":288,"Index":288,"Attempt":0,"Launch Time":1427397516762,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":283,"Index":283,"Attempt":0,"Launch Time":1427397516748,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516762,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":761089,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":289,"Index":289,"Attempt":0,"Launch Time":1427397516764,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":284,"Index":284,"Attempt":0,"Launch Time":1427397516750,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516764,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1384071,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":290,"Index":290,"Attempt":0,"Launch Time":1427397516766,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":275,"Index":275,"Attempt":0,"Launch Time":1427397516730,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516766,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":6,"Executor Run Time":25,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":2674775,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":291,"Index":291,"Attempt":0,"Launch Time":1427397516770,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":285,"Index":285,"Attempt":0,"Launch Time":1427397516758,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516771,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1468412,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":292,"Index":292,"Attempt":0,"Launch Time":1427397516776,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":256,"Index":256,"Attempt":0,"Launch Time":1427397516646,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516776,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":128,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":82497711,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":293,"Index":293,"Attempt":0,"Launch Time":1427397516777,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":253,"Index":253,"Attempt":0,"Launch Time":1427397516641,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516778,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":135,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":85013064,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":294,"Index":294,"Attempt":0,"Launch Time":1427397516786,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":290,"Index":290,"Attempt":0,"Launch Time":1427397516766,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516787,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1218711,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":295,"Index":295,"Attempt":0,"Launch Time":1427397516790,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":292,"Index":292,"Attempt":0,"Launch Time":1427397516776,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516790,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":904333,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":296,"Index":296,"Attempt":0,"Launch Time":1427397516794,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":293,"Index":293,"Attempt":0,"Launch Time":1427397516777,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516794,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":1532209,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":297,"Index":297,"Attempt":0,"Launch Time":1427397516798,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":294,"Index":294,"Attempt":0,"Launch Time":1427397516786,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516799,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":837720,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":298,"Index":298,"Attempt":0,"Launch Time":1427397516801,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":286,"Index":286,"Attempt":0,"Launch Time":1427397516760,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516801,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":39,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1204841,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":299,"Index":299,"Attempt":0,"Launch Time":1427397516805,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":295,"Index":295,"Attempt":0,"Launch Time":1427397516790,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516805,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":514610,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":300,"Index":300,"Attempt":0,"Launch Time":1427397516806,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":296,"Index":296,"Attempt":0,"Launch Time":1427397516794,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516806,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1007479,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":301,"Index":301,"Attempt":0,"Launch Time":1427397516819,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":297,"Index":297,"Attempt":0,"Launch Time":1427397516798,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516819,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":19,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1566994,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":302,"Index":302,"Attempt":0,"Launch Time":1427397516821,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":299,"Index":299,"Attempt":0,"Launch Time":1427397516805,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516821,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":723564,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":303,"Index":303,"Attempt":0,"Launch Time":1427397516829,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":301,"Index":301,"Attempt":0,"Launch Time":1427397516819,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516829,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":832310,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":304,"Index":304,"Attempt":0,"Launch Time":1427397516833,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":302,"Index":302,"Attempt":0,"Launch Time":1427397516821,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516833,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":948880,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":305,"Index":305,"Attempt":0,"Launch Time":1427397516834,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":288,"Index":288,"Attempt":0,"Launch Time":1427397516762,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516835,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":71,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":836825,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":306,"Index":306,"Attempt":0,"Launch Time":1427397516835,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":298,"Index":298,"Attempt":0,"Launch Time":1427397516801,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516836,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":33,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":636012,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":307,"Index":307,"Attempt":0,"Launch Time":1427397516843,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":308,"Index":308,"Attempt":0,"Launch Time":1427397516844,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":300,"Index":300,"Attempt":0,"Launch Time":1427397516806,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516844,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":36,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":604797,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":303,"Index":303,"Attempt":0,"Launch Time":1427397516829,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516844,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":987993,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":309,"Index":309,"Attempt":0,"Launch Time":1427397516846,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":304,"Index":304,"Attempt":0,"Launch Time":1427397516833,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516846,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":938716,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":310,"Index":310,"Attempt":0,"Launch Time":1427397516851,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":306,"Index":306,"Attempt":0,"Launch Time":1427397516835,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516851,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1013438,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":311,"Index":311,"Attempt":0,"Launch Time":1427397516853,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":305,"Index":305,"Attempt":0,"Launch Time":1427397516834,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516853,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":2304832,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":312,"Index":312,"Attempt":0,"Launch Time":1427397516856,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":307,"Index":307,"Attempt":0,"Launch Time":1427397516843,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516856,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":881106,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":313,"Index":313,"Attempt":0,"Launch Time":1427397516857,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":308,"Index":308,"Attempt":0,"Launch Time":1427397516844,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516857,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1149081,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":314,"Index":314,"Attempt":0,"Launch Time":1427397516859,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":309,"Index":309,"Attempt":0,"Launch Time":1427397516846,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516860,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":1227905,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":315,"Index":315,"Attempt":0,"Launch Time":1427397516868,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":312,"Index":312,"Attempt":0,"Launch Time":1427397516856,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516869,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":818296,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":316,"Index":316,"Attempt":0,"Launch Time":1427397516869,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":310,"Index":310,"Attempt":0,"Launch Time":1427397516851,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516869,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1294808,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":317,"Index":317,"Attempt":0,"Launch Time":1427397516871,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":313,"Index":313,"Attempt":0,"Launch Time":1427397516857,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516871,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1270124,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":318,"Index":318,"Attempt":0,"Launch Time":1427397516872,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":314,"Index":314,"Attempt":0,"Launch Time":1427397516859,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516872,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1007462,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":319,"Index":319,"Attempt":0,"Launch Time":1427397516881,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":311,"Index":311,"Attempt":0,"Launch Time":1427397516853,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516881,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":27,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":2263558,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":320,"Index":320,"Attempt":0,"Launch Time":1427397516882,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":315,"Index":315,"Attempt":0,"Launch Time":1427397516868,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516882,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":822692,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":321,"Index":321,"Attempt":0,"Launch Time":1427397516885,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":318,"Index":318,"Attempt":0,"Launch Time":1427397516872,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516885,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":865866,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":322,"Index":322,"Attempt":0,"Launch Time":1427397516885,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":316,"Index":316,"Attempt":0,"Launch Time":1427397516869,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516886,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":625382,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":317,"Index":317,"Attempt":0,"Launch Time":1427397516871,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516886,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":820572,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":323,"Index":323,"Attempt":0,"Launch Time":1427397516886,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":324,"Index":324,"Attempt":0,"Launch Time":1427397516901,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":289,"Index":289,"Attempt":0,"Launch Time":1427397516764,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516901,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":136,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":71692999,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":325,"Index":325,"Attempt":0,"Launch Time":1427397516914,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":324,"Index":324,"Attempt":0,"Launch Time":1427397516901,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516914,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":849191,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":326,"Index":326,"Attempt":0,"Launch Time":1427397516919,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":319,"Index":319,"Attempt":0,"Launch Time":1427397516881,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516920,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":5,"Executor Run Time":31,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":970277,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":327,"Index":327,"Attempt":0,"Launch Time":1427397516926,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":325,"Index":325,"Attempt":0,"Launch Time":1427397516914,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516926,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":1158253,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":328,"Index":328,"Attempt":0,"Launch Time":1427397516930,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":326,"Index":326,"Attempt":0,"Launch Time":1427397516919,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516931,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":861117,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":329,"Index":329,"Attempt":0,"Launch Time":1427397516938,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":327,"Index":327,"Attempt":0,"Launch Time":1427397516926,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516938,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":695280,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":330,"Index":330,"Attempt":0,"Launch Time":1427397516942,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":328,"Index":328,"Attempt":0,"Launch Time":1427397516930,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516943,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":952902,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":331,"Index":331,"Attempt":0,"Launch Time":1427397516950,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":329,"Index":329,"Attempt":0,"Launch Time":1427397516938,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516950,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":743062,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":332,"Index":332,"Attempt":0,"Launch Time":1427397516951,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":287,"Index":287,"Attempt":0,"Launch Time":1427397516761,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516951,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":189,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":61275953,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":333,"Index":333,"Attempt":0,"Launch Time":1427397516952,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":291,"Index":291,"Attempt":0,"Launch Time":1427397516770,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516953,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":181,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":2829169,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":334,"Index":334,"Attempt":0,"Launch Time":1427397516957,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":330,"Index":330,"Attempt":0,"Launch Time":1427397516942,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516957,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1859274,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":335,"Index":335,"Attempt":0,"Launch Time":1427397516965,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":331,"Index":331,"Attempt":0,"Launch Time":1427397516950,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516965,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1157612,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":336,"Index":336,"Attempt":0,"Launch Time":1427397516966,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":333,"Index":333,"Attempt":0,"Launch Time":1427397516952,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516966,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":644566,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":332,"Index":332,"Attempt":0,"Launch Time":1427397516951,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516966,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":724475,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":337,"Index":337,"Attempt":0,"Launch Time":1427397516967,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":338,"Index":338,"Attempt":0,"Launch Time":1427397516970,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":323,"Index":323,"Attempt":0,"Launch Time":1427397516886,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516970,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":83,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":62525299,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":339,"Index":339,"Attempt":0,"Launch Time":1427397516970,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":334,"Index":334,"Attempt":0,"Launch Time":1427397516957,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516971,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9666,"Shuffle Write Time":685650,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":340,"Index":340,"Attempt":0,"Launch Time":1427397516978,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":335,"Index":335,"Attempt":0,"Launch Time":1427397516965,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516978,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9650,"Shuffle Write Time":933222,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":341,"Index":341,"Attempt":0,"Launch Time":1427397516980,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":336,"Index":336,"Attempt":0,"Launch Time":1427397516966,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516980,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9648,"Shuffle Write Time":843684,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":342,"Index":342,"Attempt":0,"Launch Time":1427397516980,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":337,"Index":337,"Attempt":0,"Launch Time":1427397516967,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516980,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9651,"Shuffle Write Time":1214932,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":343,"Index":343,"Attempt":0,"Launch Time":1427397516986,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":338,"Index":338,"Attempt":0,"Launch Time":1427397516970,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516986,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9651,"Shuffle Write Time":1593390,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":344,"Index":344,"Attempt":0,"Launch Time":1427397516989,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":320,"Index":320,"Attempt":0,"Launch Time":1427397516882,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516989,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":107,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":95769818,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":345,"Index":345,"Attempt":0,"Launch Time":1427397516992,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":339,"Index":339,"Attempt":0,"Launch Time":1427397516970,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516992,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":20,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9647,"Shuffle Write Time":1314676,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":346,"Index":346,"Attempt":0,"Launch Time":1427397516993,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":341,"Index":341,"Attempt":0,"Launch Time":1427397516980,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516994,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1017580,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":347,"Index":347,"Attempt":0,"Launch Time":1427397516994,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":340,"Index":340,"Attempt":0,"Launch Time":1427397516978,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516995,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9649,"Shuffle Write Time":1095444,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":348,"Index":348,"Attempt":0,"Launch Time":1427397516998,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":342,"Index":342,"Attempt":0,"Launch Time":1427397516980,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516998,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":3855566,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":349,"Index":349,"Attempt":0,"Launch Time":1427397517000,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":343,"Index":343,"Attempt":0,"Launch Time":1427397516986,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517000,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1152773,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":350,"Index":350,"Attempt":0,"Launch Time":1427397517016,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":345,"Index":345,"Attempt":0,"Launch Time":1427397516992,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517016,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":22,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1160926,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":351,"Index":351,"Attempt":0,"Launch Time":1427397517017,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":348,"Index":348,"Attempt":0,"Launch Time":1427397516998,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517017,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":19,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1102575,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":352,"Index":352,"Attempt":0,"Launch Time":1427397517020,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":344,"Index":344,"Attempt":0,"Launch Time":1427397516989,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517020,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":29,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":2292906,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":353,"Index":353,"Attempt":0,"Launch Time":1427397517021,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":349,"Index":349,"Attempt":0,"Launch Time":1427397517000,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517021,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":1168034,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":354,"Index":354,"Attempt":0,"Launch Time":1427397517022,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":347,"Index":347,"Attempt":0,"Launch Time":1427397516994,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517023,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":27,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":807976,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":355,"Index":355,"Attempt":0,"Launch Time":1427397517029,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":322,"Index":322,"Attempt":0,"Launch Time":1427397516885,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517029,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":142,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1187593,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":356,"Index":356,"Attempt":0,"Launch Time":1427397517031,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":321,"Index":321,"Attempt":0,"Launch Time":1427397516885,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517031,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":146,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":2162370,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":357,"Index":357,"Attempt":0,"Launch Time":1427397517033,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":346,"Index":346,"Attempt":0,"Launch Time":1427397516993,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517033,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":37,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":2826934,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":358,"Index":358,"Attempt":0,"Launch Time":1427397517034,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":353,"Index":353,"Attempt":0,"Launch Time":1427397517021,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517034,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1055736,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":359,"Index":359,"Attempt":0,"Launch Time":1427397517042,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":352,"Index":352,"Attempt":0,"Launch Time":1427397517020,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517042,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":5785380,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":360,"Index":360,"Attempt":0,"Launch Time":1427397517047,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":361,"Index":361,"Attempt":0,"Launch Time":1427397517047,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":355,"Index":355,"Attempt":0,"Launch Time":1427397517029,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517047,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":3505869,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":358,"Index":358,"Attempt":0,"Launch Time":1427397517034,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517047,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1250631,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":362,"Index":362,"Attempt":0,"Launch Time":1427397517055,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":356,"Index":356,"Attempt":0,"Launch Time":1427397517031,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517055,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":23,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":5128594,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":363,"Index":363,"Attempt":0,"Launch Time":1427397517061,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":361,"Index":361,"Attempt":0,"Launch Time":1427397517047,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517061,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1167859,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":364,"Index":364,"Attempt":0,"Launch Time":1427397517061,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":360,"Index":360,"Attempt":0,"Launch Time":1427397517047,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517062,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1405558,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":365,"Index":365,"Attempt":0,"Launch Time":1427397517064,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":359,"Index":359,"Attempt":0,"Launch Time":1427397517042,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517064,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":20,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1831979,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":366,"Index":366,"Attempt":0,"Launch Time":1427397517071,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":362,"Index":362,"Attempt":0,"Launch Time":1427397517055,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517071,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1339156,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":367,"Index":367,"Attempt":0,"Launch Time":1427397517074,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":363,"Index":363,"Attempt":0,"Launch Time":1427397517061,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517074,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":749564,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":368,"Index":368,"Attempt":0,"Launch Time":1427397517077,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":364,"Index":364,"Attempt":0,"Launch Time":1427397517061,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517078,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1212245,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":369,"Index":369,"Attempt":0,"Launch Time":1427397517082,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":365,"Index":365,"Attempt":0,"Launch Time":1427397517064,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517083,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1658156,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":370,"Index":370,"Attempt":0,"Launch Time":1427397517086,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":357,"Index":357,"Attempt":0,"Launch Time":1427397517033,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517086,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":52,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":5310680,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":371,"Index":371,"Attempt":0,"Launch Time":1427397517087,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":367,"Index":367,"Attempt":0,"Launch Time":1427397517074,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517087,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1005569,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":372,"Index":372,"Attempt":0,"Launch Time":1427397517088,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":366,"Index":366,"Attempt":0,"Launch Time":1427397517071,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517088,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1612039,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":373,"Index":373,"Attempt":0,"Launch Time":1427397517091,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":368,"Index":368,"Attempt":0,"Launch Time":1427397517077,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517091,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":982290,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":374,"Index":374,"Attempt":0,"Launch Time":1427397517098,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":369,"Index":369,"Attempt":0,"Launch Time":1427397517082,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517099,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":962960,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":375,"Index":375,"Attempt":0,"Launch Time":1427397517101,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":371,"Index":371,"Attempt":0,"Launch Time":1427397517087,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517101,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1561476,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":376,"Index":376,"Attempt":0,"Launch Time":1427397517113,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":373,"Index":373,"Attempt":0,"Launch Time":1427397517091,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517114,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1089926,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":377,"Index":377,"Attempt":0,"Launch Time":1427397517117,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":372,"Index":372,"Attempt":0,"Launch Time":1427397517088,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517118,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":28,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1476305,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":378,"Index":378,"Attempt":0,"Launch Time":1427397517122,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":370,"Index":370,"Attempt":0,"Launch Time":1427397517086,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517122,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":33,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":1679617,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":379,"Index":379,"Attempt":0,"Launch Time":1427397517139,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":374,"Index":374,"Attempt":0,"Launch Time":1427397517098,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517139,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":40,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9612,"Shuffle Write Time":15480325,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":380,"Index":380,"Attempt":0,"Launch Time":1427397517140,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":375,"Index":375,"Attempt":0,"Launch Time":1427397517101,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517140,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":38,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":14202156,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":381,"Index":381,"Attempt":0,"Launch Time":1427397517141,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":376,"Index":376,"Attempt":0,"Launch Time":1427397517113,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517141,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":27,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":14597620,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":382,"Index":382,"Attempt":0,"Launch Time":1427397517145,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":377,"Index":377,"Attempt":0,"Launch Time":1427397517117,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517145,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":26,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":13160317,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":383,"Index":383,"Attempt":0,"Launch Time":1427397517149,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":350,"Index":350,"Attempt":0,"Launch Time":1427397517016,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517149,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":132,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":22313020,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":351,"Index":351,"Attempt":0,"Launch Time":1427397517017,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517150,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":131,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":36937750,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":384,"Index":384,"Attempt":0,"Launch Time":1427397517150,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":385,"Index":385,"Attempt":0,"Launch Time":1427397517162,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":386,"Index":386,"Attempt":0,"Launch Time":1427397517163,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":383,"Index":383,"Attempt":0,"Launch Time":1427397517149,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517163,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":732682,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":384,"Index":384,"Attempt":0,"Launch Time":1427397517150,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517163,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":868663,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":387,"Index":387,"Attempt":0,"Launch Time":1427397517166,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":354,"Index":354,"Attempt":0,"Launch Time":1427397517022,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517166,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":42,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":16656179,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":388,"Index":388,"Attempt":0,"Launch Time":1427397517167,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":378,"Index":378,"Attempt":0,"Launch Time":1427397517122,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517168,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":40,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":7873571,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":389,"Index":389,"Attempt":0,"Launch Time":1427397517177,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":386,"Index":386,"Attempt":0,"Launch Time":1427397517163,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517177,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1546840,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":390,"Index":390,"Attempt":0,"Launch Time":1427397517179,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":385,"Index":385,"Attempt":0,"Launch Time":1427397517162,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517179,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1098172,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":391,"Index":391,"Attempt":0,"Launch Time":1427397517180,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":387,"Index":387,"Attempt":0,"Launch Time":1427397517166,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517181,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":972993,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":392,"Index":392,"Attempt":0,"Launch Time":1427397517188,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":388,"Index":388,"Attempt":0,"Launch Time":1427397517167,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517189,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":2451862,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":393,"Index":393,"Attempt":0,"Launch Time":1427397517190,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":389,"Index":389,"Attempt":0,"Launch Time":1427397517177,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517191,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":1191798,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":394,"Index":394,"Attempt":0,"Launch Time":1427397517192,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":390,"Index":390,"Attempt":0,"Launch Time":1427397517179,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517193,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9615,"Shuffle Write Time":807803,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":395,"Index":395,"Attempt":0,"Launch Time":1427397517195,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":391,"Index":391,"Attempt":0,"Launch Time":1427397517180,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517195,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1708633,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":396,"Index":396,"Attempt":0,"Launch Time":1427397517204,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":393,"Index":393,"Attempt":0,"Launch Time":1427397517190,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517204,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1253358,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":397,"Index":397,"Attempt":0,"Launch Time":1427397517205,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":394,"Index":394,"Attempt":0,"Launch Time":1427397517192,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517205,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":773985,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":398,"Index":398,"Attempt":0,"Launch Time":1427397517207,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":392,"Index":392,"Attempt":0,"Launch Time":1427397517188,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517207,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1344108,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":399,"Index":399,"Attempt":0,"Launch Time":1427397517208,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":395,"Index":395,"Attempt":0,"Launch Time":1427397517195,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517209,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1477012,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":400,"Index":400,"Attempt":0,"Launch Time":1427397517210,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":380,"Index":380,"Attempt":0,"Launch Time":1427397517140,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517211,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":70,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":14664841,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":401,"Index":401,"Attempt":0,"Launch Time":1427397517217,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":402,"Index":402,"Attempt":0,"Launch Time":1427397517228,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":403,"Index":403,"Attempt":0,"Launch Time":1427397517228,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":396,"Index":396,"Attempt":0,"Launch Time":1427397517204,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517228,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1002641,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":399,"Index":399,"Attempt":0,"Launch Time":1427397517208,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517228,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":19,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":590442,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":397,"Index":397,"Attempt":0,"Launch Time":1427397517205,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517229,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":545047,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":404,"Index":404,"Attempt":0,"Launch Time":1427397517232,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":398,"Index":398,"Attempt":0,"Launch Time":1427397517207,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517232,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":24,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1789100,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":405,"Index":405,"Attempt":0,"Launch Time":1427397517237,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":382,"Index":382,"Attempt":0,"Launch Time":1427397517145,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517237,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":91,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9614,"Shuffle Write Time":2981032,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":406,"Index":406,"Attempt":0,"Launch Time":1427397517237,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":400,"Index":400,"Attempt":0,"Launch Time":1427397517210,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517238,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":25,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1841304,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":407,"Index":407,"Attempt":0,"Launch Time":1427397517241,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":401,"Index":401,"Attempt":0,"Launch Time":1427397517217,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517241,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1328705,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":408,"Index":408,"Attempt":0,"Launch Time":1427397517242,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":403,"Index":403,"Attempt":0,"Launch Time":1427397517228,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517243,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1814322,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":409,"Index":409,"Attempt":0,"Launch Time":1427397517245,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":402,"Index":402,"Attempt":0,"Launch Time":1427397517228,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517246,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":2628000,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":410,"Index":410,"Attempt":0,"Launch Time":1427397517252,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":404,"Index":404,"Attempt":0,"Launch Time":1427397517232,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517252,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":19,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":3967154,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":411,"Index":411,"Attempt":0,"Launch Time":1427397517254,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":407,"Index":407,"Attempt":0,"Launch Time":1427397517241,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517254,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1215156,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":412,"Index":412,"Attempt":0,"Launch Time":1427397517255,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":408,"Index":408,"Attempt":0,"Launch Time":1427397517242,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517256,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1252701,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":413,"Index":413,"Attempt":0,"Launch Time":1427397517260,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":409,"Index":409,"Attempt":0,"Launch Time":1427397517245,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517260,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1187345,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":414,"Index":414,"Attempt":0,"Launch Time":1427397517261,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":405,"Index":405,"Attempt":0,"Launch Time":1427397517237,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517261,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":19,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":3154171,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":415,"Index":415,"Attempt":0,"Launch Time":1427397517264,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":379,"Index":379,"Attempt":0,"Launch Time":1427397517139,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517264,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":122,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":4896741,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":416,"Index":416,"Attempt":0,"Launch Time":1427397517266,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":406,"Index":406,"Attempt":0,"Launch Time":1427397517237,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517266,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":27,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1972281,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":417,"Index":417,"Attempt":0,"Launch Time":1427397517267,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":411,"Index":411,"Attempt":0,"Launch Time":1427397517254,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517268,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1000609,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":418,"Index":418,"Attempt":0,"Launch Time":1427397517268,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":410,"Index":410,"Attempt":0,"Launch Time":1427397517252,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517268,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1269698,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":419,"Index":419,"Attempt":0,"Launch Time":1427397517268,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":412,"Index":412,"Attempt":0,"Launch Time":1427397517255,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517269,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":862428,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":420,"Index":420,"Attempt":0,"Launch Time":1427397517278,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":381,"Index":381,"Attempt":0,"Launch Time":1427397517141,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517278,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":136,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":78728345,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":421,"Index":421,"Attempt":0,"Launch Time":1427397517282,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":415,"Index":415,"Attempt":0,"Launch Time":1427397517264,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517282,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1819081,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":422,"Index":422,"Attempt":0,"Launch Time":1427397517290,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":420,"Index":420,"Attempt":0,"Launch Time":1427397517278,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517291,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":592846,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":423,"Index":423,"Attempt":0,"Launch Time":1427397517301,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":421,"Index":421,"Attempt":0,"Launch Time":1427397517282,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517302,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":1116267,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":424,"Index":424,"Attempt":0,"Launch Time":1427397517304,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":416,"Index":416,"Attempt":0,"Launch Time":1427397517266,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517304,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":37,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":2064986,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":425,"Index":425,"Attempt":0,"Launch Time":1427397517305,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":422,"Index":422,"Attempt":0,"Launch Time":1427397517290,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517305,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":930744,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":426,"Index":426,"Attempt":0,"Launch Time":1427397517307,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":413,"Index":413,"Attempt":0,"Launch Time":1427397517260,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517307,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":45,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":2592970,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":427,"Index":427,"Attempt":0,"Launch Time":1427397517314,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":423,"Index":423,"Attempt":0,"Launch Time":1427397517301,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517314,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":828874,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":428,"Index":428,"Attempt":0,"Launch Time":1427397517329,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":425,"Index":425,"Attempt":0,"Launch Time":1427397517305,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517329,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":621117,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":429,"Index":429,"Attempt":0,"Launch Time":1427397517332,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":424,"Index":424,"Attempt":0,"Launch Time":1427397517304,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517332,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":26,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1106143,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":430,"Index":430,"Attempt":0,"Launch Time":1427397517335,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":427,"Index":427,"Attempt":0,"Launch Time":1427397517314,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517335,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":20,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1018339,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":431,"Index":431,"Attempt":0,"Launch Time":1427397517336,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":426,"Index":426,"Attempt":0,"Launch Time":1427397517307,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517337,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":28,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":2062339,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":432,"Index":432,"Attempt":0,"Launch Time":1427397517344,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":428,"Index":428,"Attempt":0,"Launch Time":1427397517329,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517344,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1170879,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":433,"Index":433,"Attempt":0,"Launch Time":1427397517348,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":434,"Index":434,"Attempt":0,"Launch Time":1427397517348,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":430,"Index":430,"Attempt":0,"Launch Time":1427397517335,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517348,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1270765,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":429,"Index":429,"Attempt":0,"Launch Time":1427397517332,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517348,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":1533391,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":435,"Index":435,"Attempt":0,"Launch Time":1427397517354,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":418,"Index":418,"Attempt":0,"Launch Time":1427397517268,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517355,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":85,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":22223345,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":436,"Index":436,"Attempt":0,"Launch Time":1427397517357,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":432,"Index":432,"Attempt":0,"Launch Time":1427397517344,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517357,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1123679,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":437,"Index":437,"Attempt":0,"Launch Time":1427397517358,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":431,"Index":431,"Attempt":0,"Launch Time":1427397517336,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517358,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":2475096,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":438,"Index":438,"Attempt":0,"Launch Time":1427397517362,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":433,"Index":433,"Attempt":0,"Launch Time":1427397517348,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517363,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1086255,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":439,"Index":439,"Attempt":0,"Launch Time":1427397517363,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":434,"Index":434,"Attempt":0,"Launch Time":1427397517348,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517364,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1953708,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":440,"Index":440,"Attempt":0,"Launch Time":1427397517372,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":436,"Index":436,"Attempt":0,"Launch Time":1427397517357,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517372,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1512789,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":441,"Index":441,"Attempt":0,"Launch Time":1427397517373,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":435,"Index":435,"Attempt":0,"Launch Time":1427397517354,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517374,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":4674473,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":442,"Index":442,"Attempt":0,"Launch Time":1427397517375,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":438,"Index":438,"Attempt":0,"Launch Time":1427397517362,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517375,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":821201,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":443,"Index":443,"Attempt":0,"Launch Time":1427397517376,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":437,"Index":437,"Attempt":0,"Launch Time":1427397517358,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517377,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":1911090,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":444,"Index":444,"Attempt":0,"Launch Time":1427397517378,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":439,"Index":439,"Attempt":0,"Launch Time":1427397517363,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517378,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1110677,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":445,"Index":445,"Attempt":0,"Launch Time":1427397517382,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":419,"Index":419,"Attempt":0,"Launch Time":1427397517268,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517382,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":111,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":4749022,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":446,"Index":446,"Attempt":0,"Launch Time":1427397517385,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":440,"Index":440,"Attempt":0,"Launch Time":1427397517372,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517385,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":1566548,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":447,"Index":447,"Attempt":0,"Launch Time":1427397517388,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":417,"Index":417,"Attempt":0,"Launch Time":1427397517267,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517388,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":119,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":59794479,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":442,"Index":442,"Attempt":0,"Launch Time":1427397517375,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517388,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":945759,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":448,"Index":448,"Attempt":0,"Launch Time":1427397517388,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":449,"Index":449,"Attempt":0,"Launch Time":1427397517390,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":443,"Index":443,"Attempt":0,"Launch Time":1427397517376,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517390,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1451393,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":450,"Index":450,"Attempt":0,"Launch Time":1427397517392,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":451,"Index":451,"Attempt":0,"Launch Time":1427397517392,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":444,"Index":444,"Attempt":0,"Launch Time":1427397517378,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517392,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1603745,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":441,"Index":441,"Attempt":0,"Launch Time":1427397517373,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517392,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":2114817,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":452,"Index":452,"Attempt":0,"Launch Time":1427397517396,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":446,"Index":446,"Attempt":0,"Launch Time":1427397517385,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517397,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1180410,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":453,"Index":453,"Attempt":0,"Launch Time":1427397517398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":414,"Index":414,"Attempt":0,"Launch Time":1427397517261,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517399,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":137,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":57639582,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":454,"Index":454,"Attempt":0,"Launch Time":1427397517410,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":453,"Index":453,"Attempt":0,"Launch Time":1427397517398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517411,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":896715,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":455,"Index":455,"Attempt":0,"Launch Time":1427397517419,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":445,"Index":445,"Attempt":0,"Launch Time":1427397517382,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517420,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":5,"Executor Run Time":32,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9605,"Shuffle Write Time":759758,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":456,"Index":456,"Attempt":0,"Launch Time":1427397517422,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":448,"Index":448,"Attempt":0,"Launch Time":1427397517388,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517422,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":32,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1277715,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":457,"Index":457,"Attempt":0,"Launch Time":1427397517439,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":454,"Index":454,"Attempt":0,"Launch Time":1427397517410,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517439,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":19,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1485312,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":458,"Index":458,"Attempt":0,"Launch Time":1427397517448,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":450,"Index":450,"Attempt":0,"Launch Time":1427397517392,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517448,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":55,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":1569023,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":459,"Index":459,"Attempt":0,"Launch Time":1427397517452,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":456,"Index":456,"Attempt":0,"Launch Time":1427397517422,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517452,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":16,"Executor Run Time":12,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":923565,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":460,"Index":460,"Attempt":0,"Launch Time":1427397517453,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":461,"Index":461,"Attempt":0,"Launch Time":1427397517453,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":455,"Index":455,"Attempt":0,"Launch Time":1427397517419,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517454,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":33,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1717503,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":457,"Index":457,"Attempt":0,"Launch Time":1427397517439,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517454,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1074176,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":462,"Index":462,"Attempt":0,"Launch Time":1427397517462,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":449,"Index":449,"Attempt":0,"Launch Time":1427397517390,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517463,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":71,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1947813,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":463,"Index":463,"Attempt":0,"Launch Time":1427397517466,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":458,"Index":458,"Attempt":0,"Launch Time":1427397517448,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517467,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":2931621,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":464,"Index":464,"Attempt":0,"Launch Time":1427397517468,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":459,"Index":459,"Attempt":0,"Launch Time":1427397517452,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517468,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":2098780,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":465,"Index":465,"Attempt":0,"Launch Time":1427397517469,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":461,"Index":461,"Attempt":0,"Launch Time":1427397517453,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517469,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":878672,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":466,"Index":466,"Attempt":0,"Launch Time":1427397517472,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":460,"Index":460,"Attempt":0,"Launch Time":1427397517453,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517472,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1298778,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":467,"Index":467,"Attempt":0,"Launch Time":1427397517480,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":464,"Index":464,"Attempt":0,"Launch Time":1427397517468,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517480,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":950545,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":468,"Index":468,"Attempt":0,"Launch Time":1427397517482,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":469,"Index":469,"Attempt":0,"Launch Time":1427397517482,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":465,"Index":465,"Attempt":0,"Launch Time":1427397517469,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517482,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1116822,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":470,"Index":470,"Attempt":0,"Launch Time":1427397517482,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":462,"Index":462,"Attempt":0,"Launch Time":1427397517462,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517483,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":3199779,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":463,"Index":463,"Attempt":0,"Launch Time":1427397517466,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517483,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9607,"Shuffle Write Time":1651032,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":471,"Index":471,"Attempt":0,"Launch Time":1427397517485,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":466,"Index":466,"Attempt":0,"Launch Time":1427397517472,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517485,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1679057,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":472,"Index":472,"Attempt":0,"Launch Time":1427397517496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":468,"Index":468,"Attempt":0,"Launch Time":1427397517482,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517496,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1324475,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":467,"Index":467,"Attempt":0,"Launch Time":1427397517480,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517496,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1127154,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":473,"Index":473,"Attempt":0,"Launch Time":1427397517496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":474,"Index":474,"Attempt":0,"Launch Time":1427397517499,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":471,"Index":471,"Attempt":0,"Launch Time":1427397517485,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517499,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9613,"Shuffle Write Time":1414588,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":475,"Index":475,"Attempt":0,"Launch Time":1427397517502,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":451,"Index":451,"Attempt":0,"Launch Time":1427397517392,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517502,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":108,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":44143768,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":476,"Index":476,"Attempt":0,"Launch Time":1427397517504,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":470,"Index":470,"Attempt":0,"Launch Time":1427397517482,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517504,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":20,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":2894428,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":477,"Index":477,"Attempt":0,"Launch Time":1427397517511,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":469,"Index":469,"Attempt":0,"Launch Time":1427397517482,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517511,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":26,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":3328936,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":478,"Index":478,"Attempt":0,"Launch Time":1427397517512,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":473,"Index":473,"Attempt":0,"Launch Time":1427397517496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517512,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1347895,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":479,"Index":479,"Attempt":0,"Launch Time":1427397517513,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":472,"Index":472,"Attempt":0,"Launch Time":1427397517496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517513,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1948031,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":480,"Index":480,"Attempt":0,"Launch Time":1427397517516,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":474,"Index":474,"Attempt":0,"Launch Time":1427397517499,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517516,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":2632678,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":481,"Index":481,"Attempt":0,"Launch Time":1427397517531,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":452,"Index":452,"Attempt":0,"Launch Time":1427397517396,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517531,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":133,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":35069714,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":482,"Index":482,"Attempt":0,"Launch Time":1427397517532,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":447,"Index":447,"Attempt":0,"Launch Time":1427397517388,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517533,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":144,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1747600,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":483,"Index":483,"Attempt":0,"Launch Time":1427397517558,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":477,"Index":477,"Attempt":0,"Launch Time":1427397517511,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517558,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":45,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1945702,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":484,"Index":484,"Attempt":0,"Launch Time":1427397517559,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":481,"Index":481,"Attempt":0,"Launch Time":1427397517531,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517559,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":27,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1438399,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":485,"Index":485,"Attempt":0,"Launch Time":1427397517562,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":482,"Index":482,"Attempt":0,"Launch Time":1427397517532,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517562,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":27,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1777334,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":486,"Index":486,"Attempt":0,"Launch Time":1427397517564,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":475,"Index":475,"Attempt":0,"Launch Time":1427397517502,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517564,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":61,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":3432026,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":487,"Index":487,"Attempt":0,"Launch Time":1427397517566,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":476,"Index":476,"Attempt":0,"Launch Time":1427397517504,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517566,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":61,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":39636620,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":488,"Index":488,"Attempt":0,"Launch Time":1427397517572,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":484,"Index":484,"Attempt":0,"Launch Time":1427397517559,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517572,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1328408,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":489,"Index":489,"Attempt":0,"Launch Time":1427397517574,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":485,"Index":485,"Attempt":0,"Launch Time":1427397517562,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517574,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":935001,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":490,"Index":490,"Attempt":0,"Launch Time":1427397517576,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":483,"Index":483,"Attempt":0,"Launch Time":1427397517558,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517576,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1745492,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":491,"Index":491,"Attempt":0,"Launch Time":1427397517578,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":486,"Index":486,"Attempt":0,"Launch Time":1427397517564,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517578,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":759501,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":492,"Index":492,"Attempt":0,"Launch Time":1427397517585,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":488,"Index":488,"Attempt":0,"Launch Time":1427397517572,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517586,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1067417,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":493,"Index":493,"Attempt":0,"Launch Time":1427397517586,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":489,"Index":489,"Attempt":0,"Launch Time":1427397517574,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517586,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":911402,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":494,"Index":494,"Attempt":0,"Launch Time":1427397517591,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":490,"Index":490,"Attempt":0,"Launch Time":1427397517576,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517592,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1978609,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":495,"Index":495,"Attempt":0,"Launch Time":1427397517593,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":491,"Index":491,"Attempt":0,"Launch Time":1427397517578,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517593,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1918613,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":496,"Index":496,"Attempt":0,"Launch Time":1427397517595,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":487,"Index":487,"Attempt":0,"Launch Time":1427397517566,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517595,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":28,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":3093626,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":497,"Index":497,"Attempt":0,"Launch Time":1427397517599,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":492,"Index":492,"Attempt":0,"Launch Time":1427397517585,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517599,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1353839,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":498,"Index":498,"Attempt":0,"Launch Time":1427397517600,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":493,"Index":493,"Attempt":0,"Launch Time":1427397517586,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517600,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1277568,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":499,"Index":499,"Attempt":0,"Launch Time":1427397517612,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":495,"Index":495,"Attempt":0,"Launch Time":1427397517593,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517612,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1271262,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":500,"Index":500,"Attempt":0,"Launch Time":1427397517614,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":494,"Index":494,"Attempt":0,"Launch Time":1427397517591,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517614,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":2791509,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":501,"Index":501,"Attempt":0,"Launch Time":1427397517616,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":497,"Index":497,"Attempt":0,"Launch Time":1427397517599,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517616,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":2494651,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":502,"Index":502,"Attempt":0,"Launch Time":1427397517617,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":498,"Index":498,"Attempt":0,"Launch Time":1427397517600,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517617,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1547112,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":503,"Index":503,"Attempt":0,"Launch Time":1427397517619,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":480,"Index":480,"Attempt":0,"Launch Time":1427397517516,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517619,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":101,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":45142493,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":504,"Index":504,"Attempt":0,"Launch Time":1427397517621,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":496,"Index":496,"Attempt":0,"Launch Time":1427397517595,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517621,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":25,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1485327,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":505,"Index":505,"Attempt":0,"Launch Time":1427397517626,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":499,"Index":499,"Attempt":0,"Launch Time":1427397517612,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517627,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1637739,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":506,"Index":506,"Attempt":0,"Launch Time":1427397517630,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":501,"Index":501,"Attempt":0,"Launch Time":1427397517616,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517630,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":925581,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":507,"Index":507,"Attempt":0,"Launch Time":1427397517630,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":502,"Index":502,"Attempt":0,"Launch Time":1427397517617,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517630,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1169102,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":508,"Index":508,"Attempt":0,"Launch Time":1427397517632,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":500,"Index":500,"Attempt":0,"Launch Time":1427397517614,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517632,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":2860778,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":509,"Index":509,"Attempt":0,"Launch Time":1427397517639,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":505,"Index":505,"Attempt":0,"Launch Time":1427397517626,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517639,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":979974,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":510,"Index":510,"Attempt":0,"Launch Time":1427397517642,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":506,"Index":506,"Attempt":0,"Launch Time":1427397517630,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517643,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":922942,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":511,"Index":511,"Attempt":0,"Launch Time":1427397517644,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":507,"Index":507,"Attempt":0,"Launch Time":1427397517630,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517688,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":821882,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":512,"Index":512,"Attempt":0,"Launch Time":1427397517688,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":513,"Index":513,"Attempt":0,"Launch Time":1427397517689,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":514,"Index":514,"Attempt":0,"Launch Time":1427397517689,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":479,"Index":479,"Attempt":0,"Launch Time":1427397517513,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517689,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":173,"Result Size":930,"JVM GC Time":18,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":12042883,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":509,"Index":509,"Attempt":0,"Launch Time":1427397517639,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517689,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":39,"Result Size":930,"JVM GC Time":14,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":579786,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":478,"Index":478,"Attempt":0,"Launch Time":1427397517512,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517689,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":175,"Result Size":930,"JVM GC Time":18,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":77089428,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":515,"Index":515,"Attempt":0,"Launch Time":1427397517693,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":504,"Index":504,"Attempt":0,"Launch Time":1427397517621,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517694,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":71,"Result Size":930,"JVM GC Time":14,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":2167756,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":516,"Index":516,"Attempt":0,"Launch Time":1427397517696,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":503,"Index":503,"Attempt":0,"Launch Time":1427397517619,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517696,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":73,"Result Size":930,"JVM GC Time":14,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":2669207,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":517,"Index":517,"Attempt":0,"Launch Time":1427397517702,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":513,"Index":513,"Attempt":0,"Launch Time":1427397517689,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517702,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":713938,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":518,"Index":518,"Attempt":0,"Launch Time":1427397517703,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":512,"Index":512,"Attempt":0,"Launch Time":1427397517688,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517703,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1123595,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":519,"Index":519,"Attempt":0,"Launch Time":1427397517704,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":508,"Index":508,"Attempt":0,"Launch Time":1427397517632,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517705,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":71,"Result Size":930,"JVM GC Time":14,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":2084143,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":520,"Index":520,"Attempt":0,"Launch Time":1427397517708,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":511,"Index":511,"Attempt":0,"Launch Time":1427397517644,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517708,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":2772446,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":521,"Index":521,"Attempt":0,"Launch Time":1427397517710,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":516,"Index":516,"Attempt":0,"Launch Time":1427397517696,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517711,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1417141,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":522,"Index":522,"Attempt":0,"Launch Time":1427397517712,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":515,"Index":515,"Attempt":0,"Launch Time":1427397517693,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517712,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1576742,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":523,"Index":523,"Attempt":0,"Launch Time":1427397517713,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":517,"Index":517,"Attempt":0,"Launch Time":1427397517702,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517713,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":867706,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":524,"Index":524,"Attempt":0,"Launch Time":1427397517714,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":518,"Index":518,"Attempt":0,"Launch Time":1427397517703,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517715,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":849661,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":525,"Index":525,"Attempt":0,"Launch Time":1427397517725,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":521,"Index":521,"Attempt":0,"Launch Time":1427397517710,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517726,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":622848,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":526,"Index":526,"Attempt":0,"Launch Time":1427397517726,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":523,"Index":523,"Attempt":0,"Launch Time":1427397517713,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517726,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":636357,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":527,"Index":527,"Attempt":0,"Launch Time":1427397517727,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":524,"Index":524,"Attempt":0,"Launch Time":1427397517714,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517727,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":939009,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":528,"Index":528,"Attempt":0,"Launch Time":1427397517730,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":522,"Index":522,"Attempt":0,"Launch Time":1427397517712,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517730,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1322126,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":529,"Index":529,"Attempt":0,"Launch Time":1427397517731,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":519,"Index":519,"Attempt":0,"Launch Time":1427397517704,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517731,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":25,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":11354138,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":530,"Index":530,"Attempt":0,"Launch Time":1427397517738,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":520,"Index":520,"Attempt":0,"Launch Time":1427397517708,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517739,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":28,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":2552117,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":531,"Index":531,"Attempt":0,"Launch Time":1427397517739,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":526,"Index":526,"Attempt":0,"Launch Time":1427397517726,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517739,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":724623,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":532,"Index":532,"Attempt":0,"Launch Time":1427397517740,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":527,"Index":527,"Attempt":0,"Launch Time":1427397517727,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517740,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1000986,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":533,"Index":533,"Attempt":0,"Launch Time":1427397517741,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":525,"Index":525,"Attempt":0,"Launch Time":1427397517725,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517741,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1728950,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":534,"Index":534,"Attempt":0,"Launch Time":1427397517749,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":528,"Index":528,"Attempt":0,"Launch Time":1427397517730,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517749,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1537752,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":535,"Index":535,"Attempt":0,"Launch Time":1427397517752,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":531,"Index":531,"Attempt":0,"Launch Time":1427397517739,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517752,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":740608,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":536,"Index":536,"Attempt":0,"Launch Time":1427397517753,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":529,"Index":529,"Attempt":0,"Launch Time":1427397517731,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517753,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":2650037,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":537,"Index":537,"Attempt":0,"Launch Time":1427397517754,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":533,"Index":533,"Attempt":0,"Launch Time":1427397517741,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517754,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1026946,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":538,"Index":538,"Attempt":0,"Launch Time":1427397517756,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":532,"Index":532,"Attempt":0,"Launch Time":1427397517740,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517756,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1401216,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":539,"Index":539,"Attempt":0,"Launch Time":1427397517758,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":530,"Index":530,"Attempt":0,"Launch Time":1427397517738,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517758,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":3609756,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":540,"Index":540,"Attempt":0,"Launch Time":1427397517785,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":534,"Index":534,"Attempt":0,"Launch Time":1427397517749,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517785,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":35,"Result Size":930,"JVM GC Time":14,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1385805,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":541,"Index":541,"Attempt":0,"Launch Time":1427397517789,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":510,"Index":510,"Attempt":0,"Launch Time":1427397517642,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517789,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":145,"Result Size":930,"JVM GC Time":28,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":998406,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":542,"Index":542,"Attempt":0,"Launch Time":1427397517798,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":514,"Index":514,"Attempt":0,"Launch Time":1427397517689,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517798,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":46,"Result Size":930,"JVM GC Time":14,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":29675408,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":543,"Index":543,"Attempt":0,"Launch Time":1427397517801,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":536,"Index":536,"Attempt":0,"Launch Time":1427397517753,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517802,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":46,"Result Size":930,"JVM GC Time":14,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1845877,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":544,"Index":544,"Attempt":0,"Launch Time":1427397517804,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":541,"Index":541,"Attempt":0,"Launch Time":1427397517789,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517804,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1460863,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":545,"Index":545,"Attempt":0,"Launch Time":1427397517804,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":540,"Index":540,"Attempt":0,"Launch Time":1427397517785,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517804,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1974499,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":546,"Index":546,"Attempt":0,"Launch Time":1427397517812,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":542,"Index":542,"Attempt":0,"Launch Time":1427397517798,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517812,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1264019,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":547,"Index":547,"Attempt":0,"Launch Time":1427397517814,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":537,"Index":537,"Attempt":0,"Launch Time":1427397517754,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517814,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":58,"Result Size":930,"JVM GC Time":14,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1409574,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":548,"Index":548,"Attempt":0,"Launch Time":1427397517815,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":535,"Index":535,"Attempt":0,"Launch Time":1427397517752,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517815,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":62,"Result Size":930,"JVM GC Time":14,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1780476,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":549,"Index":549,"Attempt":0,"Launch Time":1427397517816,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":543,"Index":543,"Attempt":0,"Launch Time":1427397517801,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517816,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1393992,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":550,"Index":550,"Attempt":0,"Launch Time":1427397517816,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":544,"Index":544,"Attempt":0,"Launch Time":1427397517804,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517816,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":873682,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":551,"Index":551,"Attempt":0,"Launch Time":1427397517819,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":552,"Index":552,"Attempt":0,"Launch Time":1427397517819,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":538,"Index":538,"Attempt":0,"Launch Time":1427397517756,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517819,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":61,"Result Size":930,"JVM GC Time":14,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1169305,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":545,"Index":545,"Attempt":0,"Launch Time":1427397517804,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517820,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":955737,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":553,"Index":553,"Attempt":0,"Launch Time":1427397517826,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":546,"Index":546,"Attempt":0,"Launch Time":1427397517812,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517826,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":2221833,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":554,"Index":554,"Attempt":0,"Launch Time":1427397517830,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":549,"Index":549,"Attempt":0,"Launch Time":1427397517816,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517830,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1045177,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":555,"Index":555,"Attempt":0,"Launch Time":1427397517835,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":550,"Index":550,"Attempt":0,"Launch Time":1427397517816,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517835,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":966618,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":556,"Index":556,"Attempt":0,"Launch Time":1427397517836,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":551,"Index":551,"Attempt":0,"Launch Time":1427397517819,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517836,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":2494317,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":557,"Index":557,"Attempt":0,"Launch Time":1427397517839,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":558,"Index":558,"Attempt":0,"Launch Time":1427397517840,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":553,"Index":553,"Attempt":0,"Launch Time":1427397517826,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517840,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1104010,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":547,"Index":547,"Attempt":0,"Launch Time":1427397517814,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517840,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":25,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":3828304,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":559,"Index":559,"Attempt":0,"Launch Time":1427397517843,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":552,"Index":552,"Attempt":0,"Launch Time":1427397517819,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517844,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":22,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":3461056,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":560,"Index":560,"Attempt":0,"Launch Time":1427397517845,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":554,"Index":554,"Attempt":0,"Launch Time":1427397517830,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517845,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1242374,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":561,"Index":561,"Attempt":0,"Launch Time":1427397517850,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":539,"Index":539,"Attempt":0,"Launch Time":1427397517758,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517850,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":92,"Result Size":930,"JVM GC Time":14,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":45116282,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":562,"Index":562,"Attempt":0,"Launch Time":1427397517851,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":563,"Index":563,"Attempt":0,"Launch Time":1427397517851,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":556,"Index":556,"Attempt":0,"Launch Time":1427397517836,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517851,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":891521,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":555,"Index":555,"Attempt":0,"Launch Time":1427397517835,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517851,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1044145,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":564,"Index":564,"Attempt":0,"Launch Time":1427397517853,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":557,"Index":557,"Attempt":0,"Launch Time":1427397517839,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517854,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1165734,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":565,"Index":565,"Attempt":0,"Launch Time":1427397517856,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":558,"Index":558,"Attempt":0,"Launch Time":1427397517840,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517856,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1039595,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":566,"Index":566,"Attempt":0,"Launch Time":1427397517857,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":560,"Index":560,"Attempt":0,"Launch Time":1427397517845,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517858,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":901364,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":567,"Index":567,"Attempt":0,"Launch Time":1427397517859,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":548,"Index":548,"Attempt":0,"Launch Time":1427397517815,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517859,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":41,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":4263226,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":568,"Index":568,"Attempt":0,"Launch Time":1427397517863,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":559,"Index":559,"Attempt":0,"Launch Time":1427397517843,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517863,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":19,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1936077,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":569,"Index":569,"Attempt":0,"Launch Time":1427397517865,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":563,"Index":563,"Attempt":0,"Launch Time":1427397517851,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517865,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1132866,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":570,"Index":570,"Attempt":0,"Launch Time":1427397517867,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":562,"Index":562,"Attempt":0,"Launch Time":1427397517851,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517867,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":2376573,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":571,"Index":571,"Attempt":0,"Launch Time":1427397517868,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":564,"Index":564,"Attempt":0,"Launch Time":1427397517853,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517868,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":888490,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":572,"Index":572,"Attempt":0,"Launch Time":1427397517879,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":566,"Index":566,"Attempt":0,"Launch Time":1427397517857,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517879,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":19,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":837492,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":573,"Index":573,"Attempt":0,"Launch Time":1427397517884,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":565,"Index":565,"Attempt":0,"Launch Time":1427397517856,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517884,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":27,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":1247308,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":574,"Index":574,"Attempt":0,"Launch Time":1427397517887,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":561,"Index":561,"Attempt":0,"Launch Time":1427397517850,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517887,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":35,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":2283669,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":575,"Index":575,"Attempt":0,"Launch Time":1427397517893,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":570,"Index":570,"Attempt":0,"Launch Time":1427397517867,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517893,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":25,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1096851,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":576,"Index":576,"Attempt":0,"Launch Time":1427397517894,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":571,"Index":571,"Attempt":0,"Launch Time":1427397517868,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517894,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1167550,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":577,"Index":577,"Attempt":0,"Launch Time":1427397517894,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":578,"Index":578,"Attempt":0,"Launch Time":1427397517894,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":569,"Index":569,"Attempt":0,"Launch Time":1427397517865,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517895,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":28,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1779674,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":572,"Index":572,"Attempt":0,"Launch Time":1427397517879,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517895,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9614,"Shuffle Write Time":1173313,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":579,"Index":579,"Attempt":0,"Launch Time":1427397517896,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":568,"Index":568,"Attempt":0,"Launch Time":1427397517863,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517896,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":32,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1198971,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":580,"Index":580,"Attempt":0,"Launch Time":1427397517907,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":567,"Index":567,"Attempt":0,"Launch Time":1427397517859,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517908,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":47,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":2599856,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":581,"Index":581,"Attempt":0,"Launch Time":1427397517911,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":574,"Index":574,"Attempt":0,"Launch Time":1427397517887,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517912,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":21,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":2396275,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":582,"Index":582,"Attempt":0,"Launch Time":1427397517922,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":580,"Index":580,"Attempt":0,"Launch Time":1427397517907,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517922,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9611,"Shuffle Write Time":817378,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":583,"Index":583,"Attempt":0,"Launch Time":1427397517930,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":581,"Index":581,"Attempt":0,"Launch Time":1427397517911,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517930,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":4658037,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":584,"Index":584,"Attempt":0,"Launch Time":1427397517937,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":582,"Index":582,"Attempt":0,"Launch Time":1427397517922,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517937,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":694222,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":585,"Index":585,"Attempt":0,"Launch Time":1427397517939,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":573,"Index":573,"Attempt":0,"Launch Time":1427397517884,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517940,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":54,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":2198659,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":586,"Index":586,"Attempt":0,"Launch Time":1427397517945,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":583,"Index":583,"Attempt":0,"Launch Time":1427397517930,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517945,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1060856,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":587,"Index":587,"Attempt":0,"Launch Time":1427397517948,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":579,"Index":579,"Attempt":0,"Launch Time":1427397517896,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517948,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":50,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1228089,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":588,"Index":588,"Attempt":0,"Launch Time":1427397517950,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":584,"Index":584,"Attempt":0,"Launch Time":1427397517937,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517950,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1171815,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":589,"Index":589,"Attempt":0,"Launch Time":1427397517955,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":585,"Index":585,"Attempt":0,"Launch Time":1427397517939,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517955,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1039888,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":590,"Index":590,"Attempt":0,"Launch Time":1427397517957,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":586,"Index":586,"Attempt":0,"Launch Time":1427397517945,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517957,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1126608,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":591,"Index":591,"Attempt":0,"Launch Time":1427397517962,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":588,"Index":588,"Attempt":0,"Launch Time":1427397517950,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517962,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9608,"Shuffle Write Time":945936,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":592,"Index":592,"Attempt":0,"Launch Time":1427397517963,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":587,"Index":587,"Attempt":0,"Launch Time":1427397517948,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517964,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":791497,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":593,"Index":593,"Attempt":0,"Launch Time":1427397517970,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":590,"Index":590,"Attempt":0,"Launch Time":1427397517957,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517971,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1111792,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":594,"Index":594,"Attempt":0,"Launch Time":1427397517976,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":589,"Index":589,"Attempt":0,"Launch Time":1427397517955,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517976,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":19,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1326906,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":595,"Index":595,"Attempt":0,"Launch Time":1427397517977,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":591,"Index":591,"Attempt":0,"Launch Time":1427397517962,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517977,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1148382,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":596,"Index":596,"Attempt":0,"Launch Time":1427397517979,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":592,"Index":592,"Attempt":0,"Launch Time":1427397517963,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517979,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":948654,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":597,"Index":597,"Attempt":0,"Launch Time":1427397517984,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":593,"Index":593,"Attempt":0,"Launch Time":1427397517970,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517984,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1199277,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":598,"Index":598,"Attempt":0,"Launch Time":1427397517985,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":577,"Index":577,"Attempt":0,"Launch Time":1427397517894,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517986,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":90,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":3660664,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":599,"Index":599,"Attempt":0,"Launch Time":1427397517989,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":578,"Index":578,"Attempt":0,"Launch Time":1427397517894,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517989,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":93,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":60387585,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":600,"Index":600,"Attempt":0,"Launch Time":1427397517991,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":594,"Index":594,"Attempt":0,"Launch Time":1427397517976,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517991,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1896265,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":601,"Index":601,"Attempt":0,"Launch Time":1427397518005,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":595,"Index":595,"Attempt":0,"Launch Time":1427397517977,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518005,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":28,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1705638,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":602,"Index":602,"Attempt":0,"Launch Time":1427397518008,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":596,"Index":596,"Attempt":0,"Launch Time":1427397517979,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518009,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":28,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":2170830,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":603,"Index":603,"Attempt":0,"Launch Time":1427397518009,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":597,"Index":597,"Attempt":0,"Launch Time":1427397517984,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518010,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":25,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":1182619,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":604,"Index":604,"Attempt":0,"Launch Time":1427397518011,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":600,"Index":600,"Attempt":0,"Launch Time":1427397517991,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518011,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":19,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1296646,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":598,"Index":598,"Attempt":0,"Launch Time":1427397517985,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518011,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":25,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1348627,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":605,"Index":605,"Attempt":0,"Launch Time":1427397518012,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":606,"Index":606,"Attempt":0,"Launch Time":1427397518017,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":599,"Index":599,"Attempt":0,"Launch Time":1427397517989,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518017,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":28,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":2200244,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":607,"Index":607,"Attempt":0,"Launch Time":1427397518020,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":601,"Index":601,"Attempt":0,"Launch Time":1427397518005,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518020,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1589678,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":608,"Index":608,"Attempt":0,"Launch Time":1427397518023,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":603,"Index":603,"Attempt":0,"Launch Time":1427397518009,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518023,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1143496,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":609,"Index":609,"Attempt":0,"Launch Time":1427397518028,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":575,"Index":575,"Attempt":0,"Launch Time":1427397517893,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518028,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":133,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1710141,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":610,"Index":610,"Attempt":0,"Launch Time":1427397518030,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":576,"Index":576,"Attempt":0,"Launch Time":1427397517894,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518030,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":136,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":85014522,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":611,"Index":611,"Attempt":0,"Launch Time":1427397518040,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":606,"Index":606,"Attempt":0,"Launch Time":1427397518017,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518041,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":23,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1345139,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":612,"Index":612,"Attempt":0,"Launch Time":1427397518042,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":609,"Index":609,"Attempt":0,"Launch Time":1427397518028,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518042,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":890819,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":613,"Index":613,"Attempt":0,"Launch Time":1427397518044,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":610,"Index":610,"Attempt":0,"Launch Time":1427397518030,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518044,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1000938,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":614,"Index":614,"Attempt":0,"Launch Time":1427397518046,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":604,"Index":604,"Attempt":0,"Launch Time":1427397518011,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518047,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":34,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1579826,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":615,"Index":615,"Attempt":0,"Launch Time":1427397518048,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":602,"Index":602,"Attempt":0,"Launch Time":1427397518008,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518049,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":39,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":26499615,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":616,"Index":616,"Attempt":0,"Launch Time":1427397518058,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":611,"Index":611,"Attempt":0,"Launch Time":1427397518040,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518058,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1263345,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":617,"Index":617,"Attempt":0,"Launch Time":1427397518058,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":613,"Index":613,"Attempt":0,"Launch Time":1427397518044,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518058,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":967919,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":618,"Index":618,"Attempt":0,"Launch Time":1427397518059,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":612,"Index":612,"Attempt":0,"Launch Time":1427397518042,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518060,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1265857,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":619,"Index":619,"Attempt":0,"Launch Time":1427397518069,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":614,"Index":614,"Attempt":0,"Launch Time":1427397518046,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518070,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":22,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":2108801,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":620,"Index":620,"Attempt":0,"Launch Time":1427397518071,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":617,"Index":617,"Attempt":0,"Launch Time":1427397518058,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518072,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":935883,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":621,"Index":621,"Attempt":0,"Launch Time":1427397518073,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":618,"Index":618,"Attempt":0,"Launch Time":1427397518059,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518073,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":893958,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":622,"Index":622,"Attempt":0,"Launch Time":1427397518073,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":616,"Index":616,"Attempt":0,"Launch Time":1427397518058,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518074,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1553353,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":623,"Index":623,"Attempt":0,"Launch Time":1427397518082,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":607,"Index":607,"Attempt":0,"Launch Time":1427397518020,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518083,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":62,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":19894084,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":624,"Index":624,"Attempt":0,"Launch Time":1427397518085,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":620,"Index":620,"Attempt":0,"Launch Time":1427397518071,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518085,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1025996,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":625,"Index":625,"Attempt":0,"Launch Time":1427397518087,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":619,"Index":619,"Attempt":0,"Launch Time":1427397518069,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518087,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1406978,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":626,"Index":626,"Attempt":0,"Launch Time":1427397518087,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":621,"Index":621,"Attempt":0,"Launch Time":1427397518073,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518087,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1144566,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":627,"Index":627,"Attempt":0,"Launch Time":1427397518089,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":615,"Index":615,"Attempt":0,"Launch Time":1427397518048,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518090,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":40,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1749583,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":628,"Index":628,"Attempt":0,"Launch Time":1427397518090,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":622,"Index":622,"Attempt":0,"Launch Time":1427397518073,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518091,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":966380,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":629,"Index":629,"Attempt":0,"Launch Time":1427397518098,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":624,"Index":624,"Attempt":0,"Launch Time":1427397518085,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518099,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1178464,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":630,"Index":630,"Attempt":0,"Launch Time":1427397518101,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":626,"Index":626,"Attempt":0,"Launch Time":1427397518087,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518101,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1000357,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":631,"Index":631,"Attempt":0,"Launch Time":1427397518102,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":625,"Index":625,"Attempt":0,"Launch Time":1427397518087,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518102,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1223620,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":632,"Index":632,"Attempt":0,"Launch Time":1427397518114,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":628,"Index":628,"Attempt":0,"Launch Time":1427397518090,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518114,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":21,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":996216,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":633,"Index":633,"Attempt":0,"Launch Time":1427397518121,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":623,"Index":623,"Attempt":0,"Launch Time":1427397518082,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518122,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":38,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":6756900,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":629,"Index":629,"Attempt":0,"Launch Time":1427397518098,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518122,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":22,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":10260216,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":634,"Index":634,"Attempt":0,"Launch Time":1427397518122,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":635,"Index":635,"Attempt":0,"Launch Time":1427397518123,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":636,"Index":636,"Attempt":0,"Launch Time":1427397518123,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":627,"Index":627,"Attempt":0,"Launch Time":1427397518089,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518123,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":31,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":3981548,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":630,"Index":630,"Attempt":0,"Launch Time":1427397518101,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518123,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":20,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1004190,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":637,"Index":637,"Attempt":0,"Launch Time":1427397518131,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":632,"Index":632,"Attempt":0,"Launch Time":1427397518114,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518131,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1181604,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":638,"Index":638,"Attempt":0,"Launch Time":1427397518133,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":631,"Index":631,"Attempt":0,"Launch Time":1427397518102,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518133,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":29,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1519273,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":639,"Index":639,"Attempt":0,"Launch Time":1427397518136,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":636,"Index":636,"Attempt":0,"Launch Time":1427397518123,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518136,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":797823,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":640,"Index":640,"Attempt":0,"Launch Time":1427397518136,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":633,"Index":633,"Attempt":0,"Launch Time":1427397518121,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518136,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1472339,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":641,"Index":641,"Attempt":0,"Launch Time":1427397518144,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":637,"Index":637,"Attempt":0,"Launch Time":1427397518131,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518144,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1257855,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":642,"Index":642,"Attempt":0,"Launch Time":1427397518148,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":643,"Index":643,"Attempt":0,"Launch Time":1427397518148,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":639,"Index":639,"Attempt":0,"Launch Time":1427397518136,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518148,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1008363,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":640,"Index":640,"Attempt":0,"Launch Time":1427397518136,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518148,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1081518,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":644,"Index":644,"Attempt":0,"Launch Time":1427397518149,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":638,"Index":638,"Attempt":0,"Launch Time":1427397518133,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518149,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1243824,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":645,"Index":645,"Attempt":0,"Launch Time":1427397518153,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":605,"Index":605,"Attempt":0,"Launch Time":1427397518012,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518153,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":141,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1971241,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":646,"Index":646,"Attempt":0,"Launch Time":1427397518161,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":644,"Index":644,"Attempt":0,"Launch Time":1427397518149,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518161,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":815227,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":647,"Index":647,"Attempt":0,"Launch Time":1427397518167,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":648,"Index":648,"Attempt":0,"Launch Time":1427397518168,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":608,"Index":608,"Attempt":0,"Launch Time":1427397518023,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518168,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1199441,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":645,"Index":645,"Attempt":0,"Launch Time":1427397518153,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518168,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":854553,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":649,"Index":649,"Attempt":0,"Launch Time":1427397518173,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":646,"Index":646,"Attempt":0,"Launch Time":1427397518161,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518173,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":919453,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":650,"Index":650,"Attempt":0,"Launch Time":1427397518185,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":648,"Index":648,"Attempt":0,"Launch Time":1427397518168,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518185,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":2238891,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":651,"Index":651,"Attempt":0,"Launch Time":1427397518186,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":649,"Index":649,"Attempt":0,"Launch Time":1427397518173,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518186,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":802232,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":652,"Index":652,"Attempt":0,"Launch Time":1427397518188,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":647,"Index":647,"Attempt":0,"Launch Time":1427397518167,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518188,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":19,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1489241,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":653,"Index":653,"Attempt":0,"Launch Time":1427397518197,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":651,"Index":651,"Attempt":0,"Launch Time":1427397518186,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518197,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":725637,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":654,"Index":654,"Attempt":0,"Launch Time":1427397518198,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":650,"Index":650,"Attempt":0,"Launch Time":1427397518185,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518198,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":715885,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":655,"Index":655,"Attempt":0,"Launch Time":1427397518201,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":652,"Index":652,"Attempt":0,"Launch Time":1427397518188,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518201,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":942925,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":656,"Index":656,"Attempt":0,"Launch Time":1427397518208,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":653,"Index":653,"Attempt":0,"Launch Time":1427397518197,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518208,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":578479,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":657,"Index":657,"Attempt":0,"Launch Time":1427397518212,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":654,"Index":654,"Attempt":0,"Launch Time":1427397518198,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518212,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":826411,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":658,"Index":658,"Attempt":0,"Launch Time":1427397518213,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":634,"Index":634,"Attempt":0,"Launch Time":1427397518122,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518213,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":89,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":3560632,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":659,"Index":659,"Attempt":0,"Launch Time":1427397518215,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":655,"Index":655,"Attempt":0,"Launch Time":1427397518201,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518215,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1279654,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":660,"Index":660,"Attempt":0,"Launch Time":1427397518219,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":656,"Index":656,"Attempt":0,"Launch Time":1427397518208,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518219,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":756789,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":661,"Index":661,"Attempt":0,"Launch Time":1427397518227,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":659,"Index":659,"Attempt":0,"Launch Time":1427397518215,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518227,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":943039,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":662,"Index":662,"Attempt":0,"Launch Time":1427397518228,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":657,"Index":657,"Attempt":0,"Launch Time":1427397518212,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518228,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":600657,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":663,"Index":663,"Attempt":0,"Launch Time":1427397518231,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":635,"Index":635,"Attempt":0,"Launch Time":1427397518123,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518231,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":105,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":62925029,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":664,"Index":664,"Attempt":0,"Launch Time":1427397518231,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":660,"Index":660,"Attempt":0,"Launch Time":1427397518219,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518231,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":705793,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":665,"Index":665,"Attempt":0,"Launch Time":1427397518247,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":661,"Index":661,"Attempt":0,"Launch Time":1427397518227,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518247,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":17,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":655679,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":666,"Index":666,"Attempt":0,"Launch Time":1427397518257,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":664,"Index":664,"Attempt":0,"Launch Time":1427397518231,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518257,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":24,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":932495,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":667,"Index":667,"Attempt":0,"Launch Time":1427397518257,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":662,"Index":662,"Attempt":0,"Launch Time":1427397518228,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518258,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":29,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":17776927,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":668,"Index":668,"Attempt":0,"Launch Time":1427397518262,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":658,"Index":658,"Attempt":0,"Launch Time":1427397518213,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518263,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":37,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":1274355,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":669,"Index":669,"Attempt":0,"Launch Time":1427397518271,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":666,"Index":666,"Attempt":0,"Launch Time":1427397518257,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518271,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":704091,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":670,"Index":670,"Attempt":0,"Launch Time":1427397518271,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":665,"Index":665,"Attempt":0,"Launch Time":1427397518247,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518272,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":23,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":998002,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":671,"Index":671,"Attempt":0,"Launch Time":1427397518272,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":667,"Index":667,"Attempt":0,"Launch Time":1427397518257,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518272,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":893909,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":672,"Index":672,"Attempt":0,"Launch Time":1427397518284,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":663,"Index":663,"Attempt":0,"Launch Time":1427397518231,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518284,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":17,"Executor Run Time":23,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":833424,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":673,"Index":673,"Attempt":0,"Launch Time":1427397518289,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":668,"Index":668,"Attempt":0,"Launch Time":1427397518262,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518289,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":24,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":3139482,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":674,"Index":674,"Attempt":0,"Launch Time":1427397518291,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":641,"Index":641,"Attempt":0,"Launch Time":1427397518144,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518291,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":146,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":96445661,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":675,"Index":675,"Attempt":0,"Launch Time":1427397518292,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":643,"Index":643,"Attempt":0,"Launch Time":1427397518148,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518292,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":45,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":2798441,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":676,"Index":676,"Attempt":0,"Launch Time":1427397518298,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":672,"Index":672,"Attempt":0,"Launch Time":1427397518284,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518298,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":940764,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":677,"Index":677,"Attempt":0,"Launch Time":1427397518302,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":673,"Index":673,"Attempt":0,"Launch Time":1427397518289,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518302,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1678220,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":678,"Index":678,"Attempt":0,"Launch Time":1427397518318,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":675,"Index":675,"Attempt":0,"Launch Time":1427397518292,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518318,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":24,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":9873260,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":674,"Index":674,"Attempt":0,"Launch Time":1427397518291,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518318,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":26,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":9068714,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":679,"Index":679,"Attempt":0,"Launch Time":1427397518318,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":680,"Index":680,"Attempt":0,"Launch Time":1427397518320,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":676,"Index":676,"Attempt":0,"Launch Time":1427397518298,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518320,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":21,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":8198893,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":681,"Index":681,"Attempt":0,"Launch Time":1427397518326,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":677,"Index":677,"Attempt":0,"Launch Time":1427397518302,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518326,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":22,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":10627703,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":682,"Index":682,"Attempt":0,"Launch Time":1427397518329,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":642,"Index":642,"Attempt":0,"Launch Time":1427397518148,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518329,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":38,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":3,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":10668028,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":683,"Index":683,"Attempt":0,"Launch Time":1427397518334,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":680,"Index":680,"Attempt":0,"Launch Time":1427397518320,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518334,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":822562,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":684,"Index":684,"Attempt":0,"Launch Time":1427397518335,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":678,"Index":678,"Attempt":0,"Launch Time":1427397518318,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518335,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1101769,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":685,"Index":685,"Attempt":0,"Launch Time":1427397518336,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":679,"Index":679,"Attempt":0,"Launch Time":1427397518318,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518337,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1655047,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":686,"Index":686,"Attempt":0,"Launch Time":1427397518339,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":681,"Index":681,"Attempt":0,"Launch Time":1427397518326,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518340,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":869198,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":687,"Index":687,"Attempt":0,"Launch Time":1427397518345,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":682,"Index":682,"Attempt":0,"Launch Time":1427397518329,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518345,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":851806,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":688,"Index":688,"Attempt":0,"Launch Time":1427397518346,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":683,"Index":683,"Attempt":0,"Launch Time":1427397518334,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518346,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":759627,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":689,"Index":689,"Attempt":0,"Launch Time":1427397518350,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":685,"Index":685,"Attempt":0,"Launch Time":1427397518336,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518350,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":1055604,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":690,"Index":690,"Attempt":0,"Launch Time":1427397518350,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":684,"Index":684,"Attempt":0,"Launch Time":1427397518335,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518351,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1246863,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":691,"Index":691,"Attempt":0,"Launch Time":1427397518354,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":686,"Index":686,"Attempt":0,"Launch Time":1427397518339,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518354,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1089776,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":692,"Index":692,"Attempt":0,"Launch Time":1427397518358,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":688,"Index":688,"Attempt":0,"Launch Time":1427397518346,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518358,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":776475,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":693,"Index":693,"Attempt":0,"Launch Time":1427397518360,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":687,"Index":687,"Attempt":0,"Launch Time":1427397518345,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518360,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1385121,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":694,"Index":694,"Attempt":0,"Launch Time":1427397518362,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":689,"Index":689,"Attempt":0,"Launch Time":1427397518350,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518362,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1052519,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":695,"Index":695,"Attempt":0,"Launch Time":1427397518375,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":690,"Index":690,"Attempt":0,"Launch Time":1427397518350,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518375,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":23,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":901887,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":696,"Index":696,"Attempt":0,"Launch Time":1427397518376,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":692,"Index":692,"Attempt":0,"Launch Time":1427397518358,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518376,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":957042,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":697,"Index":697,"Attempt":0,"Launch Time":1427397518378,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":691,"Index":691,"Attempt":0,"Launch Time":1427397518354,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518378,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":23,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1165536,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":698,"Index":698,"Attempt":0,"Launch Time":1427397518380,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":669,"Index":669,"Attempt":0,"Launch Time":1427397518271,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518380,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":108,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":44998455,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":699,"Index":699,"Attempt":0,"Launch Time":1427397518384,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":693,"Index":693,"Attempt":0,"Launch Time":1427397518360,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518384,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":23,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":1261925,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":700,"Index":700,"Attempt":0,"Launch Time":1427397518388,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":696,"Index":696,"Attempt":0,"Launch Time":1427397518376,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518388,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":917106,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":701,"Index":701,"Attempt":0,"Launch Time":1427397518389,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":695,"Index":695,"Attempt":0,"Launch Time":1427397518375,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518389,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1332842,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":702,"Index":702,"Attempt":0,"Launch Time":1427397518390,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":694,"Index":694,"Attempt":0,"Launch Time":1427397518362,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518390,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":27,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1789898,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":703,"Index":703,"Attempt":0,"Launch Time":1427397518392,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":697,"Index":697,"Attempt":0,"Launch Time":1427397518378,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518392,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9612,"Shuffle Write Time":1368981,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":704,"Index":704,"Attempt":0,"Launch Time":1427397518406,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":705,"Index":705,"Attempt":0,"Launch Time":1427397518406,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":670,"Index":670,"Attempt":0,"Launch Time":1427397518271,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518406,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":133,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":6364313,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":698,"Index":698,"Attempt":0,"Launch Time":1427397518380,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518406,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":24,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1019458,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":671,"Index":671,"Attempt":0,"Launch Time":1427397518272,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518407,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":134,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":4275349,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":706,"Index":706,"Attempt":0,"Launch Time":1427397518407,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":707,"Index":707,"Attempt":0,"Launch Time":1427397518419,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":704,"Index":704,"Attempt":0,"Launch Time":1427397518406,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518419,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":782736,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":708,"Index":708,"Attempt":0,"Launch Time":1427397518420,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":706,"Index":706,"Attempt":0,"Launch Time":1427397518407,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518420,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":950640,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":709,"Index":709,"Attempt":0,"Launch Time":1427397518420,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":705,"Index":705,"Attempt":0,"Launch Time":1427397518406,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518421,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9615,"Shuffle Write Time":539270,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":710,"Index":710,"Attempt":0,"Launch Time":1427397518422,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":699,"Index":699,"Attempt":0,"Launch Time":1427397518384,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518423,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":37,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1752342,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":711,"Index":711,"Attempt":0,"Launch Time":1427397518432,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":707,"Index":707,"Attempt":0,"Launch Time":1427397518419,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518432,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1043565,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":712,"Index":712,"Attempt":0,"Launch Time":1427397518432,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":708,"Index":708,"Attempt":0,"Launch Time":1427397518420,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518432,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":614836,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":713,"Index":713,"Attempt":0,"Launch Time":1427397518433,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":709,"Index":709,"Attempt":0,"Launch Time":1427397518420,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518433,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":728789,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":714,"Index":714,"Attempt":0,"Launch Time":1427397518444,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":711,"Index":711,"Attempt":0,"Launch Time":1427397518432,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518444,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":777714,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":715,"Index":715,"Attempt":0,"Launch Time":1427397518446,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":716,"Index":716,"Attempt":0,"Launch Time":1427397518446,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":713,"Index":713,"Attempt":0,"Launch Time":1427397518433,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518446,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9613,"Shuffle Write Time":627763,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":712,"Index":712,"Attempt":0,"Launch Time":1427397518432,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518446,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1426603,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":717,"Index":717,"Attempt":0,"Launch Time":1427397518448,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":710,"Index":710,"Attempt":0,"Launch Time":1427397518422,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518448,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":23,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":2362929,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":718,"Index":718,"Attempt":0,"Launch Time":1427397518458,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":714,"Index":714,"Attempt":0,"Launch Time":1427397518444,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518458,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1277226,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":719,"Index":719,"Attempt":0,"Launch Time":1427397518459,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":716,"Index":716,"Attempt":0,"Launch Time":1427397518446,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518460,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1138865,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":720,"Index":720,"Attempt":0,"Launch Time":1427397518460,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":715,"Index":715,"Attempt":0,"Launch Time":1427397518446,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518460,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":804645,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":721,"Index":721,"Attempt":0,"Launch Time":1427397518467,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":717,"Index":717,"Attempt":0,"Launch Time":1427397518448,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518467,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":2278262,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":722,"Index":722,"Attempt":0,"Launch Time":1427397518470,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":701,"Index":701,"Attempt":0,"Launch Time":1427397518389,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518470,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":80,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":3457251,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":723,"Index":723,"Attempt":0,"Launch Time":1427397518473,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":719,"Index":719,"Attempt":0,"Launch Time":1427397518459,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518473,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1142979,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":724,"Index":724,"Attempt":0,"Launch Time":1427397518473,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":725,"Index":725,"Attempt":0,"Launch Time":1427397518473,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":718,"Index":718,"Attempt":0,"Launch Time":1427397518458,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518473,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1282930,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":720,"Index":720,"Attempt":0,"Launch Time":1427397518460,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518474,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1157078,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":726,"Index":726,"Attempt":0,"Launch Time":1427397518476,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":702,"Index":702,"Attempt":0,"Launch Time":1427397518390,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518476,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":85,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1628826,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":727,"Index":727,"Attempt":0,"Launch Time":1427397518499,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":723,"Index":723,"Attempt":0,"Launch Time":1427397518473,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518499,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":25,"Result Size":930,"JVM GC Time":12,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1492933,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":728,"Index":728,"Attempt":0,"Launch Time":1427397518501,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":725,"Index":725,"Attempt":0,"Launch Time":1427397518473,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518501,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":27,"Result Size":930,"JVM GC Time":12,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":15954768,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":729,"Index":729,"Attempt":0,"Launch Time":1427397518501,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":724,"Index":724,"Attempt":0,"Launch Time":1427397518473,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518502,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":27,"Result Size":930,"JVM GC Time":12,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":845647,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":730,"Index":730,"Attempt":0,"Launch Time":1427397518503,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":721,"Index":721,"Attempt":0,"Launch Time":1427397518467,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518503,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":34,"Result Size":930,"JVM GC Time":12,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":2278855,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":731,"Index":731,"Attempt":0,"Launch Time":1427397518509,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":722,"Index":722,"Attempt":0,"Launch Time":1427397518470,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518509,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":34,"Result Size":930,"JVM GC Time":12,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1318500,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":732,"Index":732,"Attempt":0,"Launch Time":1427397518509,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":726,"Index":726,"Attempt":0,"Launch Time":1427397518476,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518509,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":33,"Result Size":930,"JVM GC Time":12,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1911498,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":733,"Index":733,"Attempt":0,"Launch Time":1427397518511,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":727,"Index":727,"Attempt":0,"Launch Time":1427397518499,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518512,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":748735,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":734,"Index":734,"Attempt":0,"Launch Time":1427397518515,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":728,"Index":728,"Attempt":0,"Launch Time":1427397518501,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518515,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":934742,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":735,"Index":735,"Attempt":0,"Launch Time":1427397518515,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":729,"Index":729,"Attempt":0,"Launch Time":1427397518501,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518515,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1185458,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":736,"Index":736,"Attempt":0,"Launch Time":1427397518520,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":730,"Index":730,"Attempt":0,"Launch Time":1427397518503,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518520,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1838332,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":737,"Index":737,"Attempt":0,"Launch Time":1427397518526,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":700,"Index":700,"Attempt":0,"Launch Time":1427397518388,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518527,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":137,"Result Size":930,"JVM GC Time":12,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":124144554,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":738,"Index":738,"Attempt":0,"Launch Time":1427397518529,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":703,"Index":703,"Attempt":0,"Launch Time":1427397518392,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518529,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":137,"Result Size":930,"JVM GC Time":12,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":83709622,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":739,"Index":739,"Attempt":0,"Launch Time":1427397518532,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":732,"Index":732,"Attempt":0,"Launch Time":1427397518509,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518532,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":21,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":3204630,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":740,"Index":740,"Attempt":0,"Launch Time":1427397518533,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":731,"Index":731,"Attempt":0,"Launch Time":1427397518509,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518533,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":23,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":4633871,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":741,"Index":741,"Attempt":0,"Launch Time":1427397518541,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":737,"Index":737,"Attempt":0,"Launch Time":1427397518526,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518541,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1839177,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":742,"Index":742,"Attempt":0,"Launch Time":1427397518542,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":738,"Index":738,"Attempt":0,"Launch Time":1427397518529,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518542,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":837096,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":743,"Index":743,"Attempt":0,"Launch Time":1427397518551,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":739,"Index":739,"Attempt":0,"Launch Time":1427397518532,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518551,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1931678,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":744,"Index":744,"Attempt":0,"Launch Time":1427397518554,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":741,"Index":741,"Attempt":0,"Launch Time":1427397518541,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518555,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1546163,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":745,"Index":745,"Attempt":0,"Launch Time":1427397518557,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":742,"Index":742,"Attempt":0,"Launch Time":1427397518542,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518557,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1294777,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":746,"Index":746,"Attempt":0,"Launch Time":1427397518565,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":740,"Index":740,"Attempt":0,"Launch Time":1427397518533,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518565,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":28,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":4071116,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":747,"Index":747,"Attempt":0,"Launch Time":1427397518568,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":744,"Index":744,"Attempt":0,"Launch Time":1427397518554,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518568,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1443843,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":748,"Index":748,"Attempt":0,"Launch Time":1427397518569,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":743,"Index":743,"Attempt":0,"Launch Time":1427397518551,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518570,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1339172,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":749,"Index":749,"Attempt":0,"Launch Time":1427397518572,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":745,"Index":745,"Attempt":0,"Launch Time":1427397518557,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518572,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1124569,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":750,"Index":750,"Attempt":0,"Launch Time":1427397518575,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":736,"Index":736,"Attempt":0,"Launch Time":1427397518520,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518575,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":55,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":30819021,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":751,"Index":751,"Attempt":0,"Launch Time":1427397518585,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":747,"Index":747,"Attempt":0,"Launch Time":1427397518568,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518585,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1216389,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":752,"Index":752,"Attempt":0,"Launch Time":1427397518587,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":746,"Index":746,"Attempt":0,"Launch Time":1427397518565,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518587,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":19,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1275108,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":753,"Index":753,"Attempt":0,"Launch Time":1427397518588,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":754,"Index":754,"Attempt":0,"Launch Time":1427397518588,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":750,"Index":750,"Attempt":0,"Launch Time":1427397518575,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518589,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":915188,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":749,"Index":749,"Attempt":0,"Launch Time":1427397518572,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518589,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":2672594,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":755,"Index":755,"Attempt":0,"Launch Time":1427397518589,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":748,"Index":748,"Attempt":0,"Launch Time":1427397518569,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518590,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":19,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":4094700,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":756,"Index":756,"Attempt":0,"Launch Time":1427397518600,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":754,"Index":754,"Attempt":0,"Launch Time":1427397518588,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518600,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":750210,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":757,"Index":757,"Attempt":0,"Launch Time":1427397518601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":751,"Index":751,"Attempt":0,"Launch Time":1427397518585,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518601,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1109654,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":758,"Index":758,"Attempt":0,"Launch Time":1427397518602,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":759,"Index":759,"Attempt":0,"Launch Time":1427397518602,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":752,"Index":752,"Attempt":0,"Launch Time":1427397518587,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518603,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":2207779,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":753,"Index":753,"Attempt":0,"Launch Time":1427397518588,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518603,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":898989,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":760,"Index":760,"Attempt":0,"Launch Time":1427397518605,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":755,"Index":755,"Attempt":0,"Launch Time":1427397518589,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518605,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1237156,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":761,"Index":761,"Attempt":0,"Launch Time":1427397518619,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":756,"Index":756,"Attempt":0,"Launch Time":1427397518600,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518619,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":19,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":908289,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":762,"Index":762,"Attempt":0,"Launch Time":1427397518621,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":733,"Index":733,"Attempt":0,"Launch Time":1427397518511,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518621,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":109,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1058745,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":763,"Index":763,"Attempt":0,"Launch Time":1427397518624,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":758,"Index":758,"Attempt":0,"Launch Time":1427397518602,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518625,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":21,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":2470317,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":764,"Index":764,"Attempt":0,"Launch Time":1427397518626,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":757,"Index":757,"Attempt":0,"Launch Time":1427397518601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518626,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":24,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1687037,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":765,"Index":765,"Attempt":0,"Launch Time":1427397518630,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":761,"Index":761,"Attempt":0,"Launch Time":1427397518619,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518630,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1104025,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":766,"Index":766,"Attempt":0,"Launch Time":1427397518632,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":760,"Index":760,"Attempt":0,"Launch Time":1427397518605,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518632,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":26,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":2279316,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":767,"Index":767,"Attempt":0,"Launch Time":1427397518636,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":759,"Index":759,"Attempt":0,"Launch Time":1427397518602,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518636,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":30,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":2308802,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":768,"Index":768,"Attempt":0,"Launch Time":1427397518637,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":763,"Index":763,"Attempt":0,"Launch Time":1427397518624,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518637,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1582336,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":769,"Index":769,"Attempt":0,"Launch Time":1427397518641,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":765,"Index":765,"Attempt":0,"Launch Time":1427397518630,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518642,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":932416,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":770,"Index":770,"Attempt":0,"Launch Time":1427397518642,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":764,"Index":764,"Attempt":0,"Launch Time":1427397518626,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518642,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1362826,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":771,"Index":771,"Attempt":0,"Launch Time":1427397518650,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":734,"Index":734,"Attempt":0,"Launch Time":1427397518515,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518650,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":134,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":107479596,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":772,"Index":772,"Attempt":0,"Launch Time":1427397518651,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":735,"Index":735,"Attempt":0,"Launch Time":1427397518515,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518651,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":134,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":96079025,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":773,"Index":773,"Attempt":0,"Launch Time":1427397518652,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":766,"Index":766,"Attempt":0,"Launch Time":1427397518632,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518652,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":904156,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":774,"Index":774,"Attempt":0,"Launch Time":1427397518656,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":770,"Index":770,"Attempt":0,"Launch Time":1427397518642,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518656,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":1207802,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":775,"Index":775,"Attempt":0,"Launch Time":1427397518665,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":762,"Index":762,"Attempt":0,"Launch Time":1427397518621,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518666,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":44,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1648965,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":776,"Index":776,"Attempt":0,"Launch Time":1427397518670,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":772,"Index":772,"Attempt":0,"Launch Time":1427397518651,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518670,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1263199,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":777,"Index":777,"Attempt":0,"Launch Time":1427397518671,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":771,"Index":771,"Attempt":0,"Launch Time":1427397518650,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518671,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1350605,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":778,"Index":778,"Attempt":0,"Launch Time":1427397518672,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":774,"Index":774,"Attempt":0,"Launch Time":1427397518656,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518672,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1097031,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":779,"Index":779,"Attempt":0,"Launch Time":1427397518673,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":773,"Index":773,"Attempt":0,"Launch Time":1427397518652,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518673,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":2225170,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":780,"Index":780,"Attempt":0,"Launch Time":1427397518686,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":778,"Index":778,"Attempt":0,"Launch Time":1427397518672,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518686,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1291359,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":781,"Index":781,"Attempt":0,"Launch Time":1427397518686,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":775,"Index":775,"Attempt":0,"Launch Time":1427397518665,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518686,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":20,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1990356,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":782,"Index":782,"Attempt":0,"Launch Time":1427397518688,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":776,"Index":776,"Attempt":0,"Launch Time":1427397518670,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518688,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":1054865,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":783,"Index":783,"Attempt":0,"Launch Time":1427397518693,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":777,"Index":777,"Attempt":0,"Launch Time":1427397518671,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518693,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1631425,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":784,"Index":784,"Attempt":0,"Launch Time":1427397518699,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":779,"Index":779,"Attempt":0,"Launch Time":1427397518673,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518699,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":19,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":2096569,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":785,"Index":785,"Attempt":0,"Launch Time":1427397518700,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":780,"Index":780,"Attempt":0,"Launch Time":1427397518686,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518700,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":882714,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":786,"Index":786,"Attempt":0,"Launch Time":1427397518706,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":782,"Index":782,"Attempt":0,"Launch Time":1427397518688,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518707,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":2184213,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":787,"Index":787,"Attempt":0,"Launch Time":1427397518709,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":781,"Index":781,"Attempt":0,"Launch Time":1427397518686,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518709,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":22,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":2846770,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":788,"Index":788,"Attempt":0,"Launch Time":1427397518714,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":785,"Index":785,"Attempt":0,"Launch Time":1427397518700,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518714,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":880165,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":789,"Index":789,"Attempt":0,"Launch Time":1427397518714,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":783,"Index":783,"Attempt":0,"Launch Time":1427397518693,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518714,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":20,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1300604,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":790,"Index":790,"Attempt":0,"Launch Time":1427397518716,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":784,"Index":784,"Attempt":0,"Launch Time":1427397518699,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518716,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1535742,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":791,"Index":791,"Attempt":0,"Launch Time":1427397518721,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":786,"Index":786,"Attempt":0,"Launch Time":1427397518706,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518721,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1538497,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":792,"Index":792,"Attempt":0,"Launch Time":1427397518727,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":788,"Index":788,"Attempt":0,"Launch Time":1427397518714,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518727,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":945529,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":793,"Index":793,"Attempt":0,"Launch Time":1427397518732,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":790,"Index":790,"Attempt":0,"Launch Time":1427397518716,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518733,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1405261,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":794,"Index":794,"Attempt":0,"Launch Time":1427397518733,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":789,"Index":789,"Attempt":0,"Launch Time":1427397518714,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518733,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1107323,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":795,"Index":795,"Attempt":0,"Launch Time":1427397518757,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":793,"Index":793,"Attempt":0,"Launch Time":1427397518732,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518758,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":24,"Result Size":930,"JVM GC Time":8,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":3140894,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":796,"Index":796,"Attempt":0,"Launch Time":1427397518761,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":768,"Index":768,"Attempt":0,"Launch Time":1427397518637,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518762,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":123,"Result Size":930,"JVM GC Time":8,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":112233137,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":797,"Index":797,"Attempt":0,"Launch Time":1427397518768,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":791,"Index":791,"Attempt":0,"Launch Time":1427397518721,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518768,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":45,"Result Size":930,"JVM GC Time":8,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":922417,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":798,"Index":798,"Attempt":0,"Launch Time":1427397518769,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":792,"Index":792,"Attempt":0,"Launch Time":1427397518727,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518769,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":42,"Result Size":930,"JVM GC Time":8,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1099371,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":799,"Index":799,"Attempt":0,"Launch Time":1427397518774,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":767,"Index":767,"Attempt":0,"Launch Time":1427397518636,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518775,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":137,"Result Size":930,"JVM GC Time":8,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":793017,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":800,"Index":800,"Attempt":0,"Launch Time":1427397518783,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":787,"Index":787,"Attempt":0,"Launch Time":1427397518709,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518783,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":5559239,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":801,"Index":801,"Attempt":0,"Launch Time":1427397518785,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":769,"Index":769,"Attempt":0,"Launch Time":1427397518641,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518785,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":143,"Result Size":930,"JVM GC Time":8,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":114225589,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":802,"Index":802,"Attempt":0,"Launch Time":1427397518790,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":799,"Index":799,"Attempt":0,"Launch Time":1427397518774,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518790,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":2083689,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":803,"Index":803,"Attempt":0,"Launch Time":1427397518797,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":800,"Index":800,"Attempt":0,"Launch Time":1427397518783,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518797,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1595268,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":804,"Index":804,"Attempt":0,"Launch Time":1427397518802,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":801,"Index":801,"Attempt":0,"Launch Time":1427397518785,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518802,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1301262,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":805,"Index":805,"Attempt":0,"Launch Time":1427397518806,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":802,"Index":802,"Attempt":0,"Launch Time":1427397518790,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518806,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1706403,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":806,"Index":806,"Attempt":0,"Launch Time":1427397518808,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":796,"Index":796,"Attempt":0,"Launch Time":1427397518761,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518808,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":43,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":4205936,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":807,"Index":807,"Attempt":0,"Launch Time":1427397518810,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":803,"Index":803,"Attempt":0,"Launch Time":1427397518797,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518811,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1312456,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":808,"Index":808,"Attempt":0,"Launch Time":1427397518816,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":804,"Index":804,"Attempt":0,"Launch Time":1427397518802,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518816,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1043761,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":809,"Index":809,"Attempt":0,"Launch Time":1427397518820,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":805,"Index":805,"Attempt":0,"Launch Time":1427397518806,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518820,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":970271,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":810,"Index":810,"Attempt":0,"Launch Time":1427397518826,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":807,"Index":807,"Attempt":0,"Launch Time":1427397518810,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518826,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":2036358,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":811,"Index":811,"Attempt":0,"Launch Time":1427397518831,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":808,"Index":808,"Attempt":0,"Launch Time":1427397518816,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518832,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1027524,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":812,"Index":812,"Attempt":0,"Launch Time":1427397518832,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":806,"Index":806,"Attempt":0,"Launch Time":1427397518808,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518833,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":23,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1752067,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":813,"Index":813,"Attempt":0,"Launch Time":1427397518834,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":809,"Index":809,"Attempt":0,"Launch Time":1427397518820,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518834,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1703839,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":814,"Index":814,"Attempt":0,"Launch Time":1427397518839,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":810,"Index":810,"Attempt":0,"Launch Time":1427397518826,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518839,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1457884,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":815,"Index":815,"Attempt":0,"Launch Time":1427397518845,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":794,"Index":794,"Attempt":0,"Launch Time":1427397518733,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518846,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":111,"Result Size":930,"JVM GC Time":8,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":55781568,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":816,"Index":816,"Attempt":0,"Launch Time":1427397518846,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":811,"Index":811,"Attempt":0,"Launch Time":1427397518831,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518846,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1934016,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":817,"Index":817,"Attempt":0,"Launch Time":1427397518848,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":813,"Index":813,"Attempt":0,"Launch Time":1427397518834,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518849,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":1063968,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":818,"Index":818,"Attempt":0,"Launch Time":1427397518849,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":812,"Index":812,"Attempt":0,"Launch Time":1427397518832,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518850,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1367111,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":819,"Index":819,"Attempt":0,"Launch Time":1427397518851,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":814,"Index":814,"Attempt":0,"Launch Time":1427397518839,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518851,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":987153,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":820,"Index":820,"Attempt":0,"Launch Time":1427397518859,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":816,"Index":816,"Attempt":0,"Launch Time":1427397518846,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518860,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1355285,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":821,"Index":821,"Attempt":0,"Launch Time":1427397518863,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":817,"Index":817,"Attempt":0,"Launch Time":1427397518848,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518863,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1200886,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":822,"Index":822,"Attempt":0,"Launch Time":1427397518865,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":823,"Index":823,"Attempt":0,"Launch Time":1427397518865,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":819,"Index":819,"Attempt":0,"Launch Time":1427397518851,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518865,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1497703,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":818,"Index":818,"Attempt":0,"Launch Time":1427397518849,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518866,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1508487,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":824,"Index":824,"Attempt":0,"Launch Time":1427397518873,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":820,"Index":820,"Attempt":0,"Launch Time":1427397518859,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518873,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1799066,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":825,"Index":825,"Attempt":0,"Launch Time":1427397518876,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":821,"Index":821,"Attempt":0,"Launch Time":1427397518863,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518876,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":1195661,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":826,"Index":826,"Attempt":0,"Launch Time":1427397518878,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":823,"Index":823,"Attempt":0,"Launch Time":1427397518865,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518878,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1152157,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":827,"Index":827,"Attempt":0,"Launch Time":1427397518879,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":822,"Index":822,"Attempt":0,"Launch Time":1427397518865,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518879,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":950730,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":828,"Index":828,"Attempt":0,"Launch Time":1427397518881,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":815,"Index":815,"Attempt":0,"Launch Time":1427397518845,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518881,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":33,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":4058005,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":829,"Index":829,"Attempt":0,"Launch Time":1427397518892,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":795,"Index":795,"Attempt":0,"Launch Time":1427397518757,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518892,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1310626,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":830,"Index":830,"Attempt":0,"Launch Time":1427397518901,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":797,"Index":797,"Attempt":0,"Launch Time":1427397518768,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518901,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":132,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":9358833,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":831,"Index":831,"Attempt":0,"Launch Time":1427397518905,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":829,"Index":829,"Attempt":0,"Launch Time":1427397518892,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518905,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1580258,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":832,"Index":832,"Attempt":0,"Launch Time":1427397518908,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":798,"Index":798,"Attempt":0,"Launch Time":1427397518769,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518908,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":139,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":88878597,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":833,"Index":833,"Attempt":0,"Launch Time":1427397518919,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":830,"Index":830,"Attempt":0,"Launch Time":1427397518901,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518919,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1750894,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":834,"Index":834,"Attempt":0,"Launch Time":1427397518920,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":831,"Index":831,"Attempt":0,"Launch Time":1427397518905,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518920,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1427789,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":835,"Index":835,"Attempt":0,"Launch Time":1427397518925,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":832,"Index":832,"Attempt":0,"Launch Time":1427397518908,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518925,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1318879,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":836,"Index":836,"Attempt":0,"Launch Time":1427397518927,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":828,"Index":828,"Attempt":0,"Launch Time":1427397518881,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518928,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":46,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":8082551,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":837,"Index":837,"Attempt":0,"Launch Time":1427397518933,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":834,"Index":834,"Attempt":0,"Launch Time":1427397518920,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518933,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":993122,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":838,"Index":838,"Attempt":0,"Launch Time":1427397518937,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":833,"Index":833,"Attempt":0,"Launch Time":1427397518919,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518937,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1927955,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":839,"Index":839,"Attempt":0,"Launch Time":1427397518940,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":835,"Index":835,"Attempt":0,"Launch Time":1427397518925,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518940,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":2263830,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":840,"Index":840,"Attempt":0,"Launch Time":1427397518949,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":837,"Index":837,"Attempt":0,"Launch Time":1427397518933,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518949,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":1182333,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":841,"Index":841,"Attempt":0,"Launch Time":1427397518950,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":836,"Index":836,"Attempt":0,"Launch Time":1427397518927,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518951,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":22,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":4335355,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":842,"Index":842,"Attempt":0,"Launch Time":1427397518951,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":838,"Index":838,"Attempt":0,"Launch Time":1427397518937,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518952,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":954364,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":843,"Index":843,"Attempt":0,"Launch Time":1427397518953,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":839,"Index":839,"Attempt":0,"Launch Time":1427397518940,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518953,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1201631,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":844,"Index":844,"Attempt":0,"Launch Time":1427397518960,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":840,"Index":840,"Attempt":0,"Launch Time":1427397518949,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518960,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":906265,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":845,"Index":845,"Attempt":0,"Launch Time":1427397518965,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":842,"Index":842,"Attempt":0,"Launch Time":1427397518951,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518965,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1738240,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":841,"Index":841,"Attempt":0,"Launch Time":1427397518950,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518965,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1216948,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":846,"Index":846,"Attempt":0,"Launch Time":1427397518965,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":847,"Index":847,"Attempt":0,"Launch Time":1427397518967,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":843,"Index":843,"Attempt":0,"Launch Time":1427397518953,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518967,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1280963,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":848,"Index":848,"Attempt":0,"Launch Time":1427397518972,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":844,"Index":844,"Attempt":0,"Launch Time":1427397518960,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518972,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1296904,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":849,"Index":849,"Attempt":0,"Launch Time":1427397518978,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":846,"Index":846,"Attempt":0,"Launch Time":1427397518965,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518978,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":867400,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":850,"Index":850,"Attempt":0,"Launch Time":1427397518979,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":847,"Index":847,"Attempt":0,"Launch Time":1427397518967,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518980,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1175717,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":851,"Index":851,"Attempt":0,"Launch Time":1427397518984,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":845,"Index":845,"Attempt":0,"Launch Time":1427397518965,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518984,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1023589,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":852,"Index":852,"Attempt":0,"Launch Time":1427397518985,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":827,"Index":827,"Attempt":0,"Launch Time":1427397518879,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518985,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":105,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":61313887,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":853,"Index":853,"Attempt":0,"Launch Time":1427397518985,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":848,"Index":848,"Attempt":0,"Launch Time":1427397518972,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518985,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":688805,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":854,"Index":854,"Attempt":0,"Launch Time":1427397518993,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":849,"Index":849,"Attempt":0,"Launch Time":1427397518978,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518993,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1224563,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":855,"Index":855,"Attempt":0,"Launch Time":1427397518995,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":850,"Index":850,"Attempt":0,"Launch Time":1427397518979,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518995,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1051221,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":856,"Index":856,"Attempt":0,"Launch Time":1427397519000,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":853,"Index":853,"Attempt":0,"Launch Time":1427397518985,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519000,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":1942802,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":857,"Index":857,"Attempt":0,"Launch Time":1427397519009,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":854,"Index":854,"Attempt":0,"Launch Time":1427397518993,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519009,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1441182,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":858,"Index":858,"Attempt":0,"Launch Time":1427397519014,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":856,"Index":856,"Attempt":0,"Launch Time":1427397519000,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519014,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1974025,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":859,"Index":859,"Attempt":0,"Launch Time":1427397519014,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":855,"Index":855,"Attempt":0,"Launch Time":1427397518995,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519014,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":3056146,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":860,"Index":860,"Attempt":0,"Launch Time":1427397519021,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":851,"Index":851,"Attempt":0,"Launch Time":1427397518984,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519021,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":37,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":6308582,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":861,"Index":861,"Attempt":0,"Launch Time":1427397519024,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":824,"Index":824,"Attempt":0,"Launch Time":1427397518873,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519024,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":150,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1917666,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":862,"Index":862,"Attempt":0,"Launch Time":1427397519029,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":826,"Index":826,"Attempt":0,"Launch Time":1427397518878,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519039,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":150,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":965692,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":863,"Index":863,"Attempt":0,"Launch Time":1427397519045,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":825,"Index":825,"Attempt":0,"Launch Time":1427397518876,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519045,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":169,"Result Size":930,"JVM GC Time":13,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":80313429,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":864,"Index":864,"Attempt":0,"Launch Time":1427397519050,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":852,"Index":852,"Attempt":0,"Launch Time":1427397518985,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519050,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":63,"Result Size":930,"JVM GC Time":8,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":2529511,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":865,"Index":865,"Attempt":0,"Launch Time":1427397519059,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":863,"Index":863,"Attempt":0,"Launch Time":1427397519045,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519059,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1177505,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":866,"Index":866,"Attempt":0,"Launch Time":1427397519062,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":861,"Index":861,"Attempt":0,"Launch Time":1427397519024,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519062,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":35,"Result Size":930,"JVM GC Time":8,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1290395,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":867,"Index":867,"Attempt":0,"Launch Time":1427397519063,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":864,"Index":864,"Attempt":0,"Launch Time":1427397519050,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519063,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1113045,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":868,"Index":868,"Attempt":0,"Launch Time":1427397519067,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":862,"Index":862,"Attempt":0,"Launch Time":1427397519029,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519067,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1950693,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":869,"Index":869,"Attempt":0,"Launch Time":1427397519071,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":865,"Index":865,"Attempt":0,"Launch Time":1427397519059,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519071,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1117194,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":870,"Index":870,"Attempt":0,"Launch Time":1427397519074,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":866,"Index":866,"Attempt":0,"Launch Time":1427397519062,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519074,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1217767,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":871,"Index":871,"Attempt":0,"Launch Time":1427397519075,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":860,"Index":860,"Attempt":0,"Launch Time":1427397519021,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519076,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":53,"Result Size":930,"JVM GC Time":8,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":4879142,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":872,"Index":872,"Attempt":0,"Launch Time":1427397519076,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":867,"Index":867,"Attempt":0,"Launch Time":1427397519063,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519077,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1065814,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":873,"Index":873,"Attempt":0,"Launch Time":1427397519080,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":868,"Index":868,"Attempt":0,"Launch Time":1427397519067,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519081,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1742448,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":874,"Index":874,"Attempt":0,"Launch Time":1427397519084,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":869,"Index":869,"Attempt":0,"Launch Time":1427397519071,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519084,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":1087908,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":875,"Index":875,"Attempt":0,"Launch Time":1427397519087,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":870,"Index":870,"Attempt":0,"Launch Time":1427397519074,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519087,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1590139,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":876,"Index":876,"Attempt":0,"Launch Time":1427397519097,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":872,"Index":872,"Attempt":0,"Launch Time":1427397519076,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519098,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":20,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":7000798,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":877,"Index":877,"Attempt":0,"Launch Time":1427397519103,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":873,"Index":873,"Attempt":0,"Launch Time":1427397519080,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519103,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":22,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":8261893,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":878,"Index":878,"Attempt":0,"Launch Time":1427397519104,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":874,"Index":874,"Attempt":0,"Launch Time":1427397519084,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519105,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":19,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":7813253,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":879,"Index":879,"Attempt":0,"Launch Time":1427397519106,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":875,"Index":875,"Attempt":0,"Launch Time":1427397519087,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519106,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":19,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":5494837,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":880,"Index":880,"Attempt":0,"Launch Time":1427397519109,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":876,"Index":876,"Attempt":0,"Launch Time":1427397519097,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519110,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":894716,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":881,"Index":881,"Attempt":0,"Launch Time":1427397519115,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":871,"Index":871,"Attempt":0,"Launch Time":1427397519075,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519116,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":39,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":10207502,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":882,"Index":882,"Attempt":0,"Launch Time":1427397519116,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":878,"Index":878,"Attempt":0,"Launch Time":1427397519104,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519117,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":884149,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":883,"Index":883,"Attempt":0,"Launch Time":1427397519119,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":884,"Index":884,"Attempt":0,"Launch Time":1427397519122,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":880,"Index":880,"Attempt":0,"Launch Time":1427397519109,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519122,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":769454,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":885,"Index":885,"Attempt":0,"Launch Time":1427397519130,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":882,"Index":882,"Attempt":0,"Launch Time":1427397519116,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519130,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":934314,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":879,"Index":879,"Attempt":0,"Launch Time":1427397519106,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519130,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":920887,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":886,"Index":886,"Attempt":0,"Launch Time":1427397519131,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":881,"Index":881,"Attempt":0,"Launch Time":1427397519115,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519131,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1003615,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":887,"Index":887,"Attempt":0,"Launch Time":1427397519132,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":883,"Index":883,"Attempt":0,"Launch Time":1427397519119,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519132,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1219642,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":888,"Index":888,"Attempt":0,"Launch Time":1427397519137,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":858,"Index":858,"Attempt":0,"Launch Time":1427397519014,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519137,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":122,"Result Size":930,"JVM GC Time":8,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":4918663,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":889,"Index":889,"Attempt":0,"Launch Time":1427397519138,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":857,"Index":857,"Attempt":0,"Launch Time":1427397519009,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519138,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":127,"Result Size":930,"JVM GC Time":8,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":26559974,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":890,"Index":890,"Attempt":0,"Launch Time":1427397519139,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":884,"Index":884,"Attempt":0,"Launch Time":1427397519122,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519139,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":795694,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":891,"Index":891,"Attempt":0,"Launch Time":1427397519142,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":885,"Index":885,"Attempt":0,"Launch Time":1427397519130,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519143,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":879291,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":892,"Index":892,"Attempt":0,"Launch Time":1427397519144,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":877,"Index":877,"Attempt":0,"Launch Time":1427397519103,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519145,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":27,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1515766,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":887,"Index":887,"Attempt":0,"Launch Time":1427397519132,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519145,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1184134,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":893,"Index":893,"Attempt":0,"Launch Time":1427397519145,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":894,"Index":894,"Attempt":0,"Launch Time":1427397519149,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":859,"Index":859,"Attempt":0,"Launch Time":1427397519014,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519150,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":133,"Result Size":930,"JVM GC Time":8,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":13382710,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":895,"Index":895,"Attempt":0,"Launch Time":1427397519163,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":889,"Index":889,"Attempt":0,"Launch Time":1427397519138,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519163,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":922664,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":896,"Index":896,"Attempt":0,"Launch Time":1427397519165,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":894,"Index":894,"Attempt":0,"Launch Time":1427397519149,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519166,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":794150,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":897,"Index":897,"Attempt":0,"Launch Time":1427397519166,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":888,"Index":888,"Attempt":0,"Launch Time":1427397519137,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519166,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1308678,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":898,"Index":898,"Attempt":0,"Launch Time":1427397519185,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":895,"Index":895,"Attempt":0,"Launch Time":1427397519163,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519185,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1006447,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":899,"Index":899,"Attempt":0,"Launch Time":1427397519188,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":896,"Index":896,"Attempt":0,"Launch Time":1427397519165,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519189,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1575459,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":900,"Index":900,"Attempt":0,"Launch Time":1427397519197,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":898,"Index":898,"Attempt":0,"Launch Time":1427397519185,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519197,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1148129,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":901,"Index":901,"Attempt":0,"Launch Time":1427397519204,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":899,"Index":899,"Attempt":0,"Launch Time":1427397519188,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519204,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":2151250,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":902,"Index":902,"Attempt":0,"Launch Time":1427397519212,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":900,"Index":900,"Attempt":0,"Launch Time":1427397519197,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519212,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":906787,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":903,"Index":903,"Attempt":0,"Launch Time":1427397519217,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":901,"Index":901,"Attempt":0,"Launch Time":1427397519204,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519217,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1383964,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":904,"Index":904,"Attempt":0,"Launch Time":1427397519221,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":897,"Index":897,"Attempt":0,"Launch Time":1427397519166,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519221,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":52,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1178254,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":905,"Index":905,"Attempt":0,"Launch Time":1427397519224,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":893,"Index":893,"Attempt":0,"Launch Time":1427397519145,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519225,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":78,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":13126730,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":906,"Index":906,"Attempt":0,"Launch Time":1427397519227,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":902,"Index":902,"Attempt":0,"Launch Time":1427397519212,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519227,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1890599,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":907,"Index":907,"Attempt":0,"Launch Time":1427397519231,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":903,"Index":903,"Attempt":0,"Launch Time":1427397519217,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519231,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9608,"Shuffle Write Time":1499259,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":908,"Index":908,"Attempt":0,"Launch Time":1427397519233,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":886,"Index":886,"Attempt":0,"Launch Time":1427397519131,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519233,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":100,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":5805222,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":909,"Index":909,"Attempt":0,"Launch Time":1427397519234,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":904,"Index":904,"Attempt":0,"Launch Time":1427397519221,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519234,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":905666,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":910,"Index":910,"Attempt":0,"Launch Time":1427397519238,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":906,"Index":906,"Attempt":0,"Launch Time":1427397519227,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519239,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1425421,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":911,"Index":911,"Attempt":0,"Launch Time":1427397519241,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":905,"Index":905,"Attempt":0,"Launch Time":1427397519224,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519241,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1165764,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":912,"Index":912,"Attempt":0,"Launch Time":1427397519243,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":907,"Index":907,"Attempt":0,"Launch Time":1427397519231,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519243,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":710233,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":913,"Index":913,"Attempt":0,"Launch Time":1427397519246,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":909,"Index":909,"Attempt":0,"Launch Time":1427397519234,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519246,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":1029149,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":914,"Index":914,"Attempt":0,"Launch Time":1427397519250,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":910,"Index":910,"Attempt":0,"Launch Time":1427397519238,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519250,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1440044,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":915,"Index":915,"Attempt":0,"Launch Time":1427397519257,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":912,"Index":912,"Attempt":0,"Launch Time":1427397519243,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519257,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1749249,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":916,"Index":916,"Attempt":0,"Launch Time":1427397519260,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":913,"Index":913,"Attempt":0,"Launch Time":1427397519246,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519260,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1174355,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":917,"Index":917,"Attempt":0,"Launch Time":1427397519262,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":918,"Index":918,"Attempt":0,"Launch Time":1427397519263,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":911,"Index":911,"Attempt":0,"Launch Time":1427397519241,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519263,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":20,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1553845,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":914,"Index":914,"Attempt":0,"Launch Time":1427397519250,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519263,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1317904,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":919,"Index":919,"Attempt":0,"Launch Time":1427397519264,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":890,"Index":890,"Attempt":0,"Launch Time":1427397519139,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519264,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":125,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1963146,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":920,"Index":920,"Attempt":0,"Launch Time":1427397519265,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":908,"Index":908,"Attempt":0,"Launch Time":1427397519233,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519265,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":32,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":8876284,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":921,"Index":921,"Attempt":0,"Launch Time":1427397519270,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":915,"Index":915,"Attempt":0,"Launch Time":1427397519257,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519270,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1215728,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":922,"Index":922,"Attempt":0,"Launch Time":1427397519273,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":916,"Index":916,"Attempt":0,"Launch Time":1427397519260,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519273,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1076483,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":923,"Index":923,"Attempt":0,"Launch Time":1427397519278,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":891,"Index":891,"Attempt":0,"Launch Time":1427397519142,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519279,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":135,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":112037179,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":924,"Index":924,"Attempt":0,"Launch Time":1427397519279,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":892,"Index":892,"Attempt":0,"Launch Time":1427397519144,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519280,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":134,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":65342858,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":925,"Index":925,"Attempt":0,"Launch Time":1427397519280,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":920,"Index":920,"Attempt":0,"Launch Time":1427397519265,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519281,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":2541286,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":926,"Index":926,"Attempt":0,"Launch Time":1427397519284,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":919,"Index":919,"Attempt":0,"Launch Time":1427397519264,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519284,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1072626,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":927,"Index":927,"Attempt":0,"Launch Time":1427397519294,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":925,"Index":925,"Attempt":0,"Launch Time":1427397519280,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519295,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1202529,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":928,"Index":928,"Attempt":0,"Launch Time":1427397519301,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":923,"Index":923,"Attempt":0,"Launch Time":1427397519278,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519301,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1798236,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":929,"Index":929,"Attempt":0,"Launch Time":1427397519301,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":924,"Index":924,"Attempt":0,"Launch Time":1427397519279,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519301,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1607043,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":930,"Index":930,"Attempt":0,"Launch Time":1427397519304,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":926,"Index":926,"Attempt":0,"Launch Time":1427397519284,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519304,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":19,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1237148,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":931,"Index":931,"Attempt":0,"Launch Time":1427397519312,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":927,"Index":927,"Attempt":0,"Launch Time":1427397519294,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519312,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1487969,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":932,"Index":932,"Attempt":0,"Launch Time":1427397519314,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":918,"Index":918,"Attempt":0,"Launch Time":1427397519263,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519314,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":50,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":25308838,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":933,"Index":933,"Attempt":0,"Launch Time":1427397519315,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":928,"Index":928,"Attempt":0,"Launch Time":1427397519301,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519316,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1097581,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":934,"Index":934,"Attempt":0,"Launch Time":1427397519338,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":929,"Index":929,"Attempt":0,"Launch Time":1427397519301,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519339,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":35,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":2645126,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":935,"Index":935,"Attempt":0,"Launch Time":1427397519340,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":930,"Index":930,"Attempt":0,"Launch Time":1427397519304,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519340,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":35,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1505657,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":936,"Index":936,"Attempt":0,"Launch Time":1427397519345,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":931,"Index":931,"Attempt":0,"Launch Time":1427397519312,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519345,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":33,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":854520,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":937,"Index":937,"Attempt":0,"Launch Time":1427397519347,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":933,"Index":933,"Attempt":0,"Launch Time":1427397519315,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519347,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":31,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1013305,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":938,"Index":938,"Attempt":0,"Launch Time":1427397519354,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":934,"Index":934,"Attempt":0,"Launch Time":1427397519338,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519354,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1443023,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":939,"Index":939,"Attempt":0,"Launch Time":1427397519355,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":935,"Index":935,"Attempt":0,"Launch Time":1427397519340,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519355,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1933750,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":940,"Index":940,"Attempt":0,"Launch Time":1427397519357,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":932,"Index":932,"Attempt":0,"Launch Time":1427397519314,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519358,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":43,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1918124,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":936,"Index":936,"Attempt":0,"Launch Time":1427397519345,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519358,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1195916,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":941,"Index":941,"Attempt":0,"Launch Time":1427397519358,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":942,"Index":942,"Attempt":0,"Launch Time":1427397519361,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":937,"Index":937,"Attempt":0,"Launch Time":1427397519347,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519361,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1342989,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":943,"Index":943,"Attempt":0,"Launch Time":1427397519367,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":938,"Index":938,"Attempt":0,"Launch Time":1427397519354,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519367,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":817165,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":944,"Index":944,"Attempt":0,"Launch Time":1427397519368,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":939,"Index":939,"Attempt":0,"Launch Time":1427397519355,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519368,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1009118,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":945,"Index":945,"Attempt":0,"Launch Time":1427397519371,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":940,"Index":940,"Attempt":0,"Launch Time":1427397519357,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519372,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1475243,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":946,"Index":946,"Attempt":0,"Launch Time":1427397519374,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":941,"Index":941,"Attempt":0,"Launch Time":1427397519358,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519375,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":2299128,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":947,"Index":947,"Attempt":0,"Launch Time":1427397519376,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":942,"Index":942,"Attempt":0,"Launch Time":1427397519361,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519376,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":2450414,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":948,"Index":948,"Attempt":0,"Launch Time":1427397519381,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":943,"Index":943,"Attempt":0,"Launch Time":1427397519367,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519381,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1585642,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":949,"Index":949,"Attempt":0,"Launch Time":1427397519382,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":944,"Index":944,"Attempt":0,"Launch Time":1427397519368,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519382,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1093025,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":950,"Index":950,"Attempt":0,"Launch Time":1427397519385,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":945,"Index":945,"Attempt":0,"Launch Time":1427397519371,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519386,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1184163,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":951,"Index":951,"Attempt":0,"Launch Time":1427397519388,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":946,"Index":946,"Attempt":0,"Launch Time":1427397519374,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519389,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1777402,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":952,"Index":952,"Attempt":0,"Launch Time":1427397519389,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":947,"Index":947,"Attempt":0,"Launch Time":1427397519376,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519389,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1305569,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":953,"Index":953,"Attempt":0,"Launch Time":1427397519396,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":948,"Index":948,"Attempt":0,"Launch Time":1427397519381,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519396,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1084889,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":954,"Index":954,"Attempt":0,"Launch Time":1427397519398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":949,"Index":949,"Attempt":0,"Launch Time":1427397519382,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519398,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":1373943,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":955,"Index":955,"Attempt":0,"Launch Time":1427397519401,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":917,"Index":917,"Attempt":0,"Launch Time":1427397519262,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519402,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":138,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1342133,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":956,"Index":956,"Attempt":0,"Launch Time":1427397519404,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":922,"Index":922,"Attempt":0,"Launch Time":1427397519273,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519404,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":19,"Executor Run Time":68,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":5965962,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":957,"Index":957,"Attempt":0,"Launch Time":1427397519417,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":921,"Index":921,"Attempt":0,"Launch Time":1427397519270,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519417,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":146,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1382575,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":958,"Index":958,"Attempt":0,"Launch Time":1427397519417,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":956,"Index":956,"Attempt":0,"Launch Time":1427397519404,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519418,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1225133,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":959,"Index":959,"Attempt":0,"Launch Time":1427397519422,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":955,"Index":955,"Attempt":0,"Launch Time":1427397519401,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519422,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1378843,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":960,"Index":960,"Attempt":0,"Launch Time":1427397519429,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":957,"Index":957,"Attempt":0,"Launch Time":1427397519417,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519430,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1248427,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":961,"Index":961,"Attempt":0,"Launch Time":1427397519431,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":958,"Index":958,"Attempt":0,"Launch Time":1427397519417,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519432,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":840158,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":962,"Index":962,"Attempt":0,"Launch Time":1427397519437,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":959,"Index":959,"Attempt":0,"Launch Time":1427397519422,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519437,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1598225,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":963,"Index":963,"Attempt":0,"Launch Time":1427397519442,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":960,"Index":960,"Attempt":0,"Launch Time":1427397519429,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519442,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":882797,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":964,"Index":964,"Attempt":0,"Launch Time":1427397519447,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":961,"Index":961,"Attempt":0,"Launch Time":1427397519431,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519447,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":2003577,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":965,"Index":965,"Attempt":0,"Launch Time":1427397519453,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":963,"Index":963,"Attempt":0,"Launch Time":1427397519442,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519453,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1212515,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":966,"Index":966,"Attempt":0,"Launch Time":1427397519454,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":962,"Index":962,"Attempt":0,"Launch Time":1427397519437,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519455,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1077814,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":967,"Index":967,"Attempt":0,"Launch Time":1427397519462,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":964,"Index":964,"Attempt":0,"Launch Time":1427397519447,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519462,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1405353,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":968,"Index":968,"Attempt":0,"Launch Time":1427397519467,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":965,"Index":965,"Attempt":0,"Launch Time":1427397519453,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519467,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":798357,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":969,"Index":969,"Attempt":0,"Launch Time":1427397519482,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":950,"Index":950,"Attempt":0,"Launch Time":1427397519385,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519482,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":96,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1706590,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":970,"Index":970,"Attempt":0,"Launch Time":1427397519491,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":968,"Index":968,"Attempt":0,"Launch Time":1427397519467,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519491,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":23,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":3033800,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":971,"Index":971,"Attempt":0,"Launch Time":1427397519501,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":966,"Index":966,"Attempt":0,"Launch Time":1427397519454,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519501,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":45,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":29708714,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":972,"Index":972,"Attempt":0,"Launch Time":1427397519504,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":970,"Index":970,"Attempt":0,"Launch Time":1427397519491,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519504,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1158147,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":973,"Index":973,"Attempt":0,"Launch Time":1427397519508,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":967,"Index":967,"Attempt":0,"Launch Time":1427397519462,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519508,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":45,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1435798,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":974,"Index":974,"Attempt":0,"Launch Time":1427397519514,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":951,"Index":951,"Attempt":0,"Launch Time":1427397519388,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519514,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":122,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":66091869,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":975,"Index":975,"Attempt":0,"Launch Time":1427397519516,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":972,"Index":972,"Attempt":0,"Launch Time":1427397519504,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519516,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1278260,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":976,"Index":976,"Attempt":0,"Launch Time":1427397519518,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":971,"Index":971,"Attempt":0,"Launch Time":1427397519501,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519519,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1822579,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":977,"Index":977,"Attempt":0,"Launch Time":1427397519526,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":952,"Index":952,"Attempt":0,"Launch Time":1427397519389,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519527,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":136,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":100350714,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":978,"Index":978,"Attempt":0,"Launch Time":1427397519530,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":953,"Index":953,"Attempt":0,"Launch Time":1427397519396,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519530,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":134,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":104511846,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":979,"Index":979,"Attempt":0,"Launch Time":1427397519533,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":974,"Index":974,"Attempt":0,"Launch Time":1427397519514,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519533,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1387747,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":980,"Index":980,"Attempt":0,"Launch Time":1427397519540,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":977,"Index":977,"Attempt":0,"Launch Time":1427397519526,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519540,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1978782,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":981,"Index":981,"Attempt":0,"Launch Time":1427397519543,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":954,"Index":954,"Attempt":0,"Launch Time":1427397519398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519543,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":145,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":96706049,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":982,"Index":982,"Attempt":0,"Launch Time":1427397519544,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":978,"Index":978,"Attempt":0,"Launch Time":1427397519530,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519544,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1339015,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":983,"Index":983,"Attempt":0,"Launch Time":1427397519546,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":979,"Index":979,"Attempt":0,"Launch Time":1427397519533,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519547,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1120076,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":984,"Index":984,"Attempt":0,"Launch Time":1427397519553,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":980,"Index":980,"Attempt":0,"Launch Time":1427397519540,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519553,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1200804,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":985,"Index":985,"Attempt":0,"Launch Time":1427397519560,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":982,"Index":982,"Attempt":0,"Launch Time":1427397519544,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519560,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1810000,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":986,"Index":986,"Attempt":0,"Launch Time":1427397519561,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":983,"Index":983,"Attempt":0,"Launch Time":1427397519546,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519561,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1767969,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":987,"Index":987,"Attempt":0,"Launch Time":1427397519563,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":981,"Index":981,"Attempt":0,"Launch Time":1427397519543,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519563,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":20,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1171712,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":988,"Index":988,"Attempt":0,"Launch Time":1427397519567,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":984,"Index":984,"Attempt":0,"Launch Time":1427397519553,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519567,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1703771,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":989,"Index":989,"Attempt":0,"Launch Time":1427397519569,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":969,"Index":969,"Attempt":0,"Launch Time":1427397519482,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519569,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":76,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":2271587,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":990,"Index":990,"Attempt":0,"Launch Time":1427397519572,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":985,"Index":985,"Attempt":0,"Launch Time":1427397519560,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519572,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1425232,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":991,"Index":991,"Attempt":0,"Launch Time":1427397519572,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":986,"Index":986,"Attempt":0,"Launch Time":1427397519561,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519573,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1133122,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":992,"Index":992,"Attempt":0,"Launch Time":1427397519578,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":987,"Index":987,"Attempt":0,"Launch Time":1427397519563,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519578,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1185286,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":993,"Index":993,"Attempt":0,"Launch Time":1427397519580,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":988,"Index":988,"Attempt":0,"Launch Time":1427397519567,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519581,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1989353,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":994,"Index":994,"Attempt":0,"Launch Time":1427397519585,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":991,"Index":991,"Attempt":0,"Launch Time":1427397519572,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519585,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1599759,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":995,"Index":995,"Attempt":0,"Launch Time":1427397519587,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":990,"Index":990,"Attempt":0,"Launch Time":1427397519572,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519587,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":1603063,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":996,"Index":996,"Attempt":0,"Launch Time":1427397519593,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":993,"Index":993,"Attempt":0,"Launch Time":1427397519580,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519593,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":840060,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":997,"Index":997,"Attempt":0,"Launch Time":1427397519596,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":989,"Index":989,"Attempt":0,"Launch Time":1427397519569,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519597,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":27,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":5183705,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":998,"Index":998,"Attempt":0,"Launch Time":1427397519597,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":994,"Index":994,"Attempt":0,"Launch Time":1427397519585,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519598,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1001529,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":999,"Index":999,"Attempt":0,"Launch Time":1427397519598,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":992,"Index":992,"Attempt":0,"Launch Time":1427397519578,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519599,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":2489437,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":995,"Index":995,"Attempt":0,"Launch Time":1427397519587,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519602,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1020616,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":996,"Index":996,"Attempt":0,"Launch Time":1427397519593,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519606,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1542373,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":997,"Index":997,"Attempt":0,"Launch Time":1427397519596,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519612,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1718136,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":999,"Index":999,"Attempt":0,"Launch Time":1427397519598,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519612,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1222086,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":998,"Index":998,"Attempt":0,"Launch Time":1427397519597,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519613,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1240678,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":973,"Index":973,"Attempt":0,"Launch Time":1427397519508,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519614,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":105,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1283929,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":976,"Index":976,"Attempt":0,"Launch Time":1427397519518,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519622,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":102,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":69851795,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":975,"Index":975,"Attempt":0,"Launch Time":1427397519516,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519624,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":107,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":94564853,"Shuffle Records Written":100}}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"map at :21","Number of Tasks":1000,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:287)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:21)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:28)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:30)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.(:32)\n$line15.$read$$iwC$$iwC$$iwC.(:34)\n$line15.$read$$iwC$$iwC.(:36)\n$line15.$read$$iwC.(:38)\n$line15.$read.(:40)\n$line15.$read$.(:44)\n$line15.$read$.()\n$line15.$eval$.(:7)\n$line15.$eval$.()\n$line15.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Submission Time":1427397514574,"Completion Time":1427397519624,"Accumulables":[]}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"replStringOf at :10","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":50,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"scala.runtime.ScalaRunTime$.replStringOf(ScalaRunTime.scala:337)\n$line19.$eval$.(:10)\n$line19.$eval$.()\n$line19.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1338)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:840)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:871)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:819)\norg.apache.spark.repl.SparkILoop.reallyInterpret$1(SparkILoop.scala:856)\norg.apache.spark.repl.SparkILoop.interpretStartingWith(SparkILoop.scala:901)\norg.apache.spark.repl.SparkILoop.command(SparkILoop.scala:813)\norg.apache.spark.repl.SparkILoop.processLine$1(SparkILoop.scala:656)\norg.apache.spark.repl.SparkILoop.innerLoop$1(SparkILoop.scala:664)\norg.apache.spark.repl.SparkILoop.org$apache$spark$repl$SparkILoop$$loop(SparkILoop.scala:669)\norg.apache.spark.repl.SparkILoop$$anonfun$org$apache$spark$repl$SparkILoop$$process$1.apply$mcZ$sp(SparkILoop.scala:996)","Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":1000,"Index":0,"Attempt":0,"Launch Time":1427397519638,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1000,"Index":0,"Attempt":0,"Launch Time":1427397519638,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519918,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":275,"Result Size":1060,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":1000,"Fetch Wait Time":4,"Remote Bytes Read":0,"Local Bytes Read":192026,"Total Records Read":2000}}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"replStringOf at :10","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":50,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"scala.runtime.ScalaRunTime$.replStringOf(ScalaRunTime.scala:337)\n$line19.$eval$.(:10)\n$line19.$eval$.()\n$line19.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1338)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:840)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:871)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:819)\norg.apache.spark.repl.SparkILoop.reallyInterpret$1(SparkILoop.scala:856)\norg.apache.spark.repl.SparkILoop.interpretStartingWith(SparkILoop.scala:901)\norg.apache.spark.repl.SparkILoop.command(SparkILoop.scala:813)\norg.apache.spark.repl.SparkILoop.processLine$1(SparkILoop.scala:656)\norg.apache.spark.repl.SparkILoop.innerLoop$1(SparkILoop.scala:664)\norg.apache.spark.repl.SparkILoop.org$apache$spark$repl$SparkILoop$$loop(SparkILoop.scala:669)\norg.apache.spark.repl.SparkILoop$$anonfun$org$apache$spark$repl$SparkILoop$$process$1.apply$mcZ$sp(SparkILoop.scala:996)","Submission Time":1427397519638,"Completion Time":1427397519919,"Accumulables":[]}} -{"Event":"SparkListenerJobEnd","Job ID":0,"Completion Time":1427397519923,"Job Result":{"Result":"JobSucceeded"}} -{"Event":"SparkListenerJobStart","Job ID":1,"Submission Time":1427397526089,"Stage Infos":[{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"map at :21","Number of Tasks":1000,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:287)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:21)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:28)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:30)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.(:32)\n$line15.$read$$iwC$$iwC$$iwC.(:34)\n$line15.$read$$iwC$$iwC.(:36)\n$line15.$read$$iwC.(:38)\n$line15.$read.(:40)\n$line15.$read$.(:44)\n$line15.$read$.()\n$line15.$eval$.(:7)\n$line15.$eval$.()\n$line15.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]},{"Stage ID":3,"Stage Attempt ID":0,"Stage Name":"next at :26","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":50,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"scala.collection.Iterator$$anon$13.next(Iterator.scala:372)\n$line22.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line22.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:31)\n$line22.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:33)\n$line22.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:35)\n$line22.$read$$iwC$$iwC$$iwC$$iwC.(:37)\n$line22.$read$$iwC$$iwC$$iwC.(:39)\n$line22.$read$$iwC$$iwC.(:41)\n$line22.$read$$iwC.(:43)\n$line22.$read.(:45)\n$line22.$read$.(:49)\n$line22.$read$.()\n$line22.$eval$.(:7)\n$line22.$eval$.()\n$line22.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]}],"Stage IDs":[2,3]} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":3,"Stage Attempt ID":0,"Stage Name":"next at :26","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":50,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"scala.collection.Iterator$$anon$13.next(Iterator.scala:372)\n$line22.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line22.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:31)\n$line22.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:33)\n$line22.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:35)\n$line22.$read$$iwC$$iwC$$iwC$$iwC.(:37)\n$line22.$read$$iwC$$iwC$$iwC.(:39)\n$line22.$read$$iwC$$iwC.(:41)\n$line22.$read$$iwC.(:43)\n$line22.$read.(:45)\n$line22.$read$.(:49)\n$line22.$read$.()\n$line22.$eval$.(:7)\n$line22.$eval$.()\n$line22.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":3,"Stage Attempt ID":0,"Task Info":{"Task ID":1001,"Index":0,"Attempt":0,"Launch Time":1427397526091,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":3,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1001,"Index":0,"Attempt":0,"Launch Time":1427397526091,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397526183,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":91,"Result Size":1060,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":1000,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":192021,"Total Records Read":2000}}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":3,"Stage Attempt ID":0,"Stage Name":"next at :26","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":50,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"scala.collection.Iterator$$anon$13.next(Iterator.scala:372)\n$line22.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line22.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:31)\n$line22.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:33)\n$line22.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:35)\n$line22.$read$$iwC$$iwC$$iwC$$iwC.(:37)\n$line22.$read$$iwC$$iwC$$iwC.(:39)\n$line22.$read$$iwC$$iwC.(:41)\n$line22.$read$$iwC.(:43)\n$line22.$read.(:45)\n$line22.$read$.(:49)\n$line22.$read$.()\n$line22.$eval$.(:7)\n$line22.$eval$.()\n$line22.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Submission Time":1427397526091,"Completion Time":1427397526184,"Accumulables":[]}} -{"Event":"SparkListenerJobEnd","Job ID":1,"Completion Time":1427397526184,"Job Result":{"Result":"JobSucceeded"}} -{"Event":"SparkListenerJobStart","Job ID":2,"Submission Time":1427397532114,"Stage Infos":[{"Stage ID":5,"Stage Attempt ID":0,"Stage Name":"next at :26","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":50,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"scala.collection.Iterator$$anon$13.next(Iterator.scala:372)\n$line24.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line24.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:31)\n$line24.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:33)\n$line24.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:35)\n$line24.$read$$iwC$$iwC$$iwC$$iwC.(:37)\n$line24.$read$$iwC$$iwC$$iwC.(:39)\n$line24.$read$$iwC$$iwC.(:41)\n$line24.$read$$iwC.(:43)\n$line24.$read.(:45)\n$line24.$read$.(:49)\n$line24.$read$.()\n$line24.$eval$.(:7)\n$line24.$eval$.()\n$line24.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]},{"Stage ID":4,"Stage Attempt ID":0,"Stage Name":"map at :21","Number of Tasks":1000,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:287)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:21)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:28)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:30)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.(:32)\n$line15.$read$$iwC$$iwC$$iwC.(:34)\n$line15.$read$$iwC$$iwC.(:36)\n$line15.$read$$iwC.(:38)\n$line15.$read.(:40)\n$line15.$read$.(:44)\n$line15.$read$.()\n$line15.$eval$.(:7)\n$line15.$eval$.()\n$line15.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]}],"Stage IDs":[5,4]} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":4,"Stage Attempt ID":0,"Stage Name":"map at :21","Number of Tasks":1000,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:287)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:21)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:28)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:30)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.(:32)\n$line15.$read$$iwC$$iwC$$iwC.(:34)\n$line15.$read$$iwC$$iwC.(:36)\n$line15.$read$$iwC.(:38)\n$line15.$read.(:40)\n$line15.$read$.(:44)\n$line15.$read$.()\n$line15.$eval$.(:7)\n$line15.$eval$.()\n$line15.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1002,"Index":0,"Attempt":0,"Launch Time":1427397532127,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1003,"Index":1,"Attempt":0,"Launch Time":1427397532128,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1004,"Index":2,"Attempt":0,"Launch Time":1427397532128,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1005,"Index":3,"Attempt":0,"Launch Time":1427397532128,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1006,"Index":4,"Attempt":0,"Launch Time":1427397532128,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1007,"Index":5,"Attempt":0,"Launch Time":1427397532128,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1008,"Index":6,"Attempt":0,"Launch Time":1427397532128,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1009,"Index":7,"Attempt":0,"Launch Time":1427397532128,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1010,"Index":8,"Attempt":0,"Launch Time":1427397532137,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1004,"Index":2,"Attempt":0,"Launch Time":1427397532128,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532138,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":324646,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1011,"Index":9,"Attempt":0,"Launch Time":1427397532138,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532147,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1012,"Index":10,"Attempt":0,"Launch Time":1427397532138,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532147,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1013,"Index":11,"Attempt":0,"Launch Time":1427397532138,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532147,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1014,"Index":12,"Attempt":0,"Launch Time":1427397532138,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532146,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1003,"Index":1,"Attempt":0,"Launch Time":1427397532128,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532139,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":305159,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1006,"Index":4,"Attempt":0,"Launch Time":1427397532128,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532139,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":517009,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1009,"Index":7,"Attempt":0,"Launch Time":1427397532128,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532139,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":305365,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1007,"Index":5,"Attempt":0,"Launch Time":1427397532128,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532139,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":409979,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1015,"Index":13,"Attempt":0,"Launch Time":1427397532139,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1005,"Index":3,"Attempt":0,"Launch Time":1427397532128,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532139,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":310558,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1016,"Index":14,"Attempt":0,"Launch Time":1427397532145,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1017,"Index":15,"Attempt":0,"Launch Time":1427397532146,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532154,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1018,"Index":16,"Attempt":0,"Launch Time":1427397532146,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532154,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1019,"Index":17,"Attempt":0,"Launch Time":1427397532146,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532156,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1014,"Index":12,"Attempt":0,"Launch Time":1427397532138,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532146,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":306103,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1010,"Index":8,"Attempt":0,"Launch Time":1427397532137,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532146,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":351670,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1020,"Index":18,"Attempt":0,"Launch Time":1427397532146,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532157,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1013,"Index":11,"Attempt":0,"Launch Time":1427397532138,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532147,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":309824,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1012,"Index":10,"Attempt":0,"Launch Time":1427397532138,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532147,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":315146,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1011,"Index":9,"Attempt":0,"Launch Time":1427397532138,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532147,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":326855,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1021,"Index":19,"Attempt":0,"Launch Time":1427397532154,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1022,"Index":20,"Attempt":0,"Launch Time":1427397532154,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532173,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1017,"Index":15,"Attempt":0,"Launch Time":1427397532146,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532154,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":371495,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1018,"Index":16,"Attempt":0,"Launch Time":1427397532146,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532154,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":341136,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1023,"Index":21,"Attempt":0,"Launch Time":1427397532155,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1019,"Index":17,"Attempt":0,"Launch Time":1427397532146,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532156,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":386762,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1024,"Index":22,"Attempt":0,"Launch Time":1427397532157,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1020,"Index":18,"Attempt":0,"Launch Time":1427397532146,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532157,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":388311,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1025,"Index":23,"Attempt":0,"Launch Time":1427397532170,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1026,"Index":24,"Attempt":0,"Launch Time":1427397532170,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1027,"Index":25,"Attempt":0,"Launch Time":1427397532172,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1002,"Index":0,"Attempt":0,"Launch Time":1427397532127,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532172,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":40,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":307919,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1022,"Index":20,"Attempt":0,"Launch Time":1427397532154,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532173,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":322177,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1016,"Index":14,"Attempt":0,"Launch Time":1427397532145,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532176,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":22,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":368695,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1028,"Index":26,"Attempt":0,"Launch Time":1427397532177,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1029,"Index":27,"Attempt":0,"Launch Time":1427397532177,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1023,"Index":21,"Attempt":0,"Launch Time":1427397532155,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532178,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":387339,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1021,"Index":19,"Attempt":0,"Launch Time":1427397532154,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532178,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":23,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":336143,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1030,"Index":28,"Attempt":0,"Launch Time":1427397532181,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1031,"Index":29,"Attempt":0,"Launch Time":1427397532181,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1032,"Index":30,"Attempt":0,"Launch Time":1427397532181,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1033,"Index":31,"Attempt":0,"Launch Time":1427397532182,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1008,"Index":6,"Attempt":0,"Launch Time":1427397532128,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532182,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":49,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":341561,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1015,"Index":13,"Attempt":0,"Launch Time":1427397532139,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532182,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":40,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":350927,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1024,"Index":22,"Attempt":0,"Launch Time":1427397532157,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532182,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":24,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":459160,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1026,"Index":24,"Attempt":0,"Launch Time":1427397532170,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532182,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":415900,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1034,"Index":32,"Attempt":0,"Launch Time":1427397532186,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1028,"Index":26,"Attempt":0,"Launch Time":1427397532177,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532186,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":311771,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1035,"Index":33,"Attempt":0,"Launch Time":1427397532186,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1029,"Index":27,"Attempt":0,"Launch Time":1427397532177,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532186,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":312091,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1036,"Index":34,"Attempt":0,"Launch Time":1427397532187,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1025,"Index":23,"Attempt":0,"Launch Time":1427397532170,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532187,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":311783,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1037,"Index":35,"Attempt":0,"Launch Time":1427397532189,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1030,"Index":28,"Attempt":0,"Launch Time":1427397532181,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532189,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":298035,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1038,"Index":36,"Attempt":0,"Launch Time":1427397532189,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1032,"Index":30,"Attempt":0,"Launch Time":1427397532181,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532190,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":303287,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1039,"Index":37,"Attempt":0,"Launch Time":1427397532191,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1033,"Index":31,"Attempt":0,"Launch Time":1427397532182,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532192,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":309664,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1040,"Index":38,"Attempt":0,"Launch Time":1427397532195,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1027,"Index":25,"Attempt":0,"Launch Time":1427397532172,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532195,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":19,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":382595,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1041,"Index":39,"Attempt":0,"Launch Time":1427397532197,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1031,"Index":29,"Attempt":0,"Launch Time":1427397532181,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532197,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":408299,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1042,"Index":40,"Attempt":0,"Launch Time":1427397532198,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1036,"Index":34,"Attempt":0,"Launch Time":1427397532187,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532198,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":364367,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1043,"Index":41,"Attempt":0,"Launch Time":1427397532201,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1039,"Index":37,"Attempt":0,"Launch Time":1427397532191,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532201,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":367495,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1044,"Index":42,"Attempt":0,"Launch Time":1427397532202,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1040,"Index":38,"Attempt":0,"Launch Time":1427397532195,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532202,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":286558,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1045,"Index":43,"Attempt":0,"Launch Time":1427397532204,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1041,"Index":39,"Attempt":0,"Launch Time":1427397532197,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532204,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":318181,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1046,"Index":44,"Attempt":0,"Launch Time":1427397532205,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1042,"Index":40,"Attempt":0,"Launch Time":1427397532198,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532205,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":282565,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1047,"Index":45,"Attempt":0,"Launch Time":1427397532209,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1044,"Index":42,"Attempt":0,"Launch Time":1427397532202,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532209,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":293888,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1048,"Index":46,"Attempt":0,"Launch Time":1427397532210,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1043,"Index":41,"Attempt":0,"Launch Time":1427397532201,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532210,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":305832,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1049,"Index":47,"Attempt":0,"Launch Time":1427397532213,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1046,"Index":44,"Attempt":0,"Launch Time":1427397532205,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532213,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":328103,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1050,"Index":48,"Attempt":0,"Launch Time":1427397532215,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1045,"Index":43,"Attempt":0,"Launch Time":1427397532204,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532215,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":335303,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1051,"Index":49,"Attempt":0,"Launch Time":1427397532217,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1047,"Index":45,"Attempt":0,"Launch Time":1427397532209,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532217,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":339035,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1052,"Index":50,"Attempt":0,"Launch Time":1427397532217,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1048,"Index":46,"Attempt":0,"Launch Time":1427397532210,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532217,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":316677,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1053,"Index":51,"Attempt":0,"Launch Time":1427397532221,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1037,"Index":35,"Attempt":0,"Launch Time":1427397532189,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532221,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":20,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":319927,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1054,"Index":52,"Attempt":0,"Launch Time":1427397532223,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1049,"Index":47,"Attempt":0,"Launch Time":1427397532213,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532223,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":360510,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1055,"Index":53,"Attempt":0,"Launch Time":1427397532224,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1056,"Index":54,"Attempt":0,"Launch Time":1427397532224,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1034,"Index":32,"Attempt":0,"Launch Time":1427397532186,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532224,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":5,"Executor Run Time":19,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":353633,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1050,"Index":48,"Attempt":0,"Launch Time":1427397532215,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532224,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":347823,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1057,"Index":55,"Attempt":0,"Launch Time":1427397532225,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1051,"Index":49,"Attempt":0,"Launch Time":1427397532217,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532225,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":379551,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1058,"Index":56,"Attempt":0,"Launch Time":1427397532225,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1052,"Index":50,"Attempt":0,"Launch Time":1427397532217,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532225,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":371448,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1059,"Index":57,"Attempt":0,"Launch Time":1427397532230,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1053,"Index":51,"Attempt":0,"Launch Time":1427397532221,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532231,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":326679,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1060,"Index":58,"Attempt":0,"Launch Time":1427397532233,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1057,"Index":55,"Attempt":0,"Launch Time":1427397532225,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532233,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":339618,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1061,"Index":59,"Attempt":0,"Launch Time":1427397532233,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1062,"Index":60,"Attempt":0,"Launch Time":1427397532233,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1054,"Index":52,"Attempt":0,"Launch Time":1427397532223,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532233,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":413134,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1055,"Index":53,"Attempt":0,"Launch Time":1427397532224,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532233,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":328512,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1063,"Index":61,"Attempt":0,"Launch Time":1427397532235,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1058,"Index":56,"Attempt":0,"Launch Time":1427397532225,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532235,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":377670,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1064,"Index":62,"Attempt":0,"Launch Time":1427397532236,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1056,"Index":54,"Attempt":0,"Launch Time":1427397532224,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532237,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":369618,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1065,"Index":63,"Attempt":0,"Launch Time":1427397532239,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1035,"Index":33,"Attempt":0,"Launch Time":1427397532186,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532239,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":33,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":3097121,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1066,"Index":64,"Attempt":0,"Launch Time":1427397532248,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1059,"Index":57,"Attempt":0,"Launch Time":1427397532230,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532248,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":294046,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1067,"Index":65,"Attempt":0,"Launch Time":1427397532255,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1061,"Index":59,"Attempt":0,"Launch Time":1427397532233,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532255,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":20,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9610,"Shuffle Write Time":320439,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1068,"Index":66,"Attempt":0,"Launch Time":1427397532255,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1062,"Index":60,"Attempt":0,"Launch Time":1427397532233,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532255,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":344267,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1069,"Index":67,"Attempt":0,"Launch Time":1427397532255,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1060,"Index":58,"Attempt":0,"Launch Time":1427397532233,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532255,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":12183325,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1070,"Index":68,"Attempt":0,"Launch Time":1427397532256,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1063,"Index":61,"Attempt":0,"Launch Time":1427397532235,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532256,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":20,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":310589,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1071,"Index":69,"Attempt":0,"Launch Time":1427397532257,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1064,"Index":62,"Attempt":0,"Launch Time":1427397532236,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532257,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":20,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":339113,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1072,"Index":70,"Attempt":0,"Launch Time":1427397532261,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1066,"Index":64,"Attempt":0,"Launch Time":1427397532248,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532261,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":327357,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1073,"Index":71,"Attempt":0,"Launch Time":1427397532263,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1068,"Index":66,"Attempt":0,"Launch Time":1427397532255,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532263,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":346438,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1074,"Index":72,"Attempt":0,"Launch Time":1427397532263,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1067,"Index":65,"Attempt":0,"Launch Time":1427397532255,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532263,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":329537,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1075,"Index":73,"Attempt":0,"Launch Time":1427397532263,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1070,"Index":68,"Attempt":0,"Launch Time":1427397532256,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532264,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":342412,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1076,"Index":74,"Attempt":0,"Launch Time":1427397532265,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1069,"Index":67,"Attempt":0,"Launch Time":1427397532255,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532265,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9614,"Shuffle Write Time":351657,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1077,"Index":75,"Attempt":0,"Launch Time":1427397532267,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1071,"Index":69,"Attempt":0,"Launch Time":1427397532257,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532267,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":306511,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1078,"Index":76,"Attempt":0,"Launch Time":1427397532270,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1072,"Index":70,"Attempt":0,"Launch Time":1427397532261,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532270,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":310319,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1079,"Index":77,"Attempt":0,"Launch Time":1427397532271,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1073,"Index":71,"Attempt":0,"Launch Time":1427397532263,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532271,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":335129,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1080,"Index":78,"Attempt":0,"Launch Time":1427397532271,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1075,"Index":73,"Attempt":0,"Launch Time":1427397532263,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532271,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":389778,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1081,"Index":79,"Attempt":0,"Launch Time":1427397532272,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1074,"Index":72,"Attempt":0,"Launch Time":1427397532263,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532272,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":335242,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1082,"Index":80,"Attempt":0,"Launch Time":1427397532276,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1076,"Index":74,"Attempt":0,"Launch Time":1427397532265,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532276,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":350986,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1083,"Index":81,"Attempt":0,"Launch Time":1427397532277,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1078,"Index":76,"Attempt":0,"Launch Time":1427397532270,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532277,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":315383,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1084,"Index":82,"Attempt":0,"Launch Time":1427397532278,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1079,"Index":77,"Attempt":0,"Launch Time":1427397532271,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532279,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":408898,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1085,"Index":83,"Attempt":0,"Launch Time":1427397532279,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1080,"Index":78,"Attempt":0,"Launch Time":1427397532271,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532280,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":368844,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1086,"Index":84,"Attempt":0,"Launch Time":1427397532280,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1038,"Index":36,"Attempt":0,"Launch Time":1427397532189,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532280,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":21,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":332193,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1087,"Index":85,"Attempt":0,"Launch Time":1427397532286,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1082,"Index":80,"Attempt":0,"Launch Time":1427397532276,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532286,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":294967,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1088,"Index":86,"Attempt":0,"Launch Time":1427397532287,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1084,"Index":82,"Attempt":0,"Launch Time":1427397532278,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532287,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":283668,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1089,"Index":87,"Attempt":0,"Launch Time":1427397532288,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1085,"Index":83,"Attempt":0,"Launch Time":1427397532279,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532288,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":634558,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1090,"Index":88,"Attempt":0,"Launch Time":1427397532288,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1081,"Index":79,"Attempt":0,"Launch Time":1427397532272,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532289,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":364350,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1091,"Index":89,"Attempt":0,"Launch Time":1427397532289,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1077,"Index":75,"Attempt":0,"Launch Time":1427397532267,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532290,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":22,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":401494,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1092,"Index":90,"Attempt":0,"Launch Time":1427397532293,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1087,"Index":85,"Attempt":0,"Launch Time":1427397532286,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532294,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":305014,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1093,"Index":91,"Attempt":0,"Launch Time":1427397532294,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1088,"Index":86,"Attempt":0,"Launch Time":1427397532287,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532294,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":274037,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1094,"Index":92,"Attempt":0,"Launch Time":1427397532297,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1089,"Index":87,"Attempt":0,"Launch Time":1427397532288,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532297,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":354671,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1095,"Index":93,"Attempt":0,"Launch Time":1427397532298,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1090,"Index":88,"Attempt":0,"Launch Time":1427397532288,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532298,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":631722,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1096,"Index":94,"Attempt":0,"Launch Time":1427397532300,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1083,"Index":81,"Attempt":0,"Launch Time":1427397532277,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532300,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":19,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":317365,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1097,"Index":95,"Attempt":0,"Launch Time":1427397532302,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1093,"Index":91,"Attempt":0,"Launch Time":1427397532294,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532302,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":313106,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1098,"Index":96,"Attempt":0,"Launch Time":1427397532302,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1099,"Index":97,"Attempt":0,"Launch Time":1427397532302,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1091,"Index":89,"Attempt":0,"Launch Time":1427397532289,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532302,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":540797,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1092,"Index":90,"Attempt":0,"Launch Time":1427397532293,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532303,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":308791,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1100,"Index":98,"Attempt":0,"Launch Time":1427397532305,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1095,"Index":93,"Attempt":0,"Launch Time":1427397532298,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532305,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":314984,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1101,"Index":99,"Attempt":0,"Launch Time":1427397532307,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1102,"Index":100,"Attempt":0,"Launch Time":1427397532307,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1096,"Index":94,"Attempt":0,"Launch Time":1427397532300,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532307,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":334169,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1094,"Index":92,"Attempt":0,"Launch Time":1427397532297,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532309,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":311798,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1103,"Index":101,"Attempt":0,"Launch Time":1427397532310,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1098,"Index":96,"Attempt":0,"Launch Time":1427397532302,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532310,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":304450,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1104,"Index":102,"Attempt":0,"Launch Time":1427397532311,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1065,"Index":63,"Attempt":0,"Launch Time":1427397532239,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532311,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":9,"Executor Run Time":50,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":311712,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1105,"Index":103,"Attempt":0,"Launch Time":1427397532312,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1097,"Index":95,"Attempt":0,"Launch Time":1427397532302,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532312,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":286888,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1106,"Index":104,"Attempt":0,"Launch Time":1427397532314,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1100,"Index":98,"Attempt":0,"Launch Time":1427397532305,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532314,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":315415,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1107,"Index":105,"Attempt":0,"Launch Time":1427397532316,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1102,"Index":100,"Attempt":0,"Launch Time":1427397532307,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532316,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":345488,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1108,"Index":106,"Attempt":0,"Launch Time":1427397532316,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1086,"Index":84,"Attempt":0,"Launch Time":1427397532280,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532316,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":35,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":364062,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1109,"Index":107,"Attempt":0,"Launch Time":1427397532328,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1104,"Index":102,"Attempt":0,"Launch Time":1427397532311,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532328,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":316963,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1110,"Index":108,"Attempt":0,"Launch Time":1427397532334,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1108,"Index":106,"Attempt":0,"Launch Time":1427397532316,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532334,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":16,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":374513,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1111,"Index":109,"Attempt":0,"Launch Time":1427397532335,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1109,"Index":107,"Attempt":0,"Launch Time":1427397532328,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532335,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":309532,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1112,"Index":110,"Attempt":0,"Launch Time":1427397532336,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1099,"Index":97,"Attempt":0,"Launch Time":1427397532302,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532336,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":32,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":369192,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1113,"Index":111,"Attempt":0,"Launch Time":1427397532341,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1110,"Index":108,"Attempt":0,"Launch Time":1427397532334,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532341,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":316258,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1114,"Index":112,"Attempt":0,"Launch Time":1427397532343,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1111,"Index":109,"Attempt":0,"Launch Time":1427397532335,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532343,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":313220,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1112,"Index":110,"Attempt":0,"Launch Time":1427397532336,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532343,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":336505,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1115,"Index":113,"Attempt":0,"Launch Time":1427397532343,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1116,"Index":114,"Attempt":0,"Launch Time":1427397532349,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1113,"Index":111,"Attempt":0,"Launch Time":1427397532341,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532349,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":320623,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1117,"Index":115,"Attempt":0,"Launch Time":1427397532351,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1114,"Index":112,"Attempt":0,"Launch Time":1427397532343,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532351,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":353178,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1118,"Index":116,"Attempt":0,"Launch Time":1427397532353,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1115,"Index":113,"Attempt":0,"Launch Time":1427397532343,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532354,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":311888,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1119,"Index":117,"Attempt":0,"Launch Time":1427397532356,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1116,"Index":114,"Attempt":0,"Launch Time":1427397532349,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532356,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":289671,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1120,"Index":118,"Attempt":0,"Launch Time":1427397532359,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1117,"Index":115,"Attempt":0,"Launch Time":1427397532351,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532359,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":324601,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1121,"Index":119,"Attempt":0,"Launch Time":1427397532363,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1118,"Index":116,"Attempt":0,"Launch Time":1427397532353,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532363,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":320545,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1122,"Index":120,"Attempt":0,"Launch Time":1427397532363,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1119,"Index":117,"Attempt":0,"Launch Time":1427397532356,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532364,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":333314,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1123,"Index":121,"Attempt":0,"Launch Time":1427397532366,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1120,"Index":118,"Attempt":0,"Launch Time":1427397532359,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532366,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":274413,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1124,"Index":122,"Attempt":0,"Launch Time":1427397532371,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1125,"Index":123,"Attempt":0,"Launch Time":1427397532374,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1126,"Index":124,"Attempt":0,"Launch Time":1427397532374,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1122,"Index":120,"Attempt":0,"Launch Time":1427397532363,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532375,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":304511,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1123,"Index":121,"Attempt":0,"Launch Time":1427397532366,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532380,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":368007,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1121,"Index":119,"Attempt":0,"Launch Time":1427397532363,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532380,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":315743,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1127,"Index":125,"Attempt":0,"Launch Time":1427397532384,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1124,"Index":122,"Attempt":0,"Launch Time":1427397532371,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532384,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":391385,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1128,"Index":126,"Attempt":0,"Launch Time":1427397532384,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1125,"Index":123,"Attempt":0,"Launch Time":1427397532374,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532384,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":350128,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1129,"Index":127,"Attempt":0,"Launch Time":1427397532387,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1126,"Index":124,"Attempt":0,"Launch Time":1427397532374,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532387,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":295821,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1130,"Index":128,"Attempt":0,"Launch Time":1427397532391,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1127,"Index":125,"Attempt":0,"Launch Time":1427397532384,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532391,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":294821,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1131,"Index":129,"Attempt":0,"Launch Time":1427397532392,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1128,"Index":126,"Attempt":0,"Launch Time":1427397532384,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532392,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":310892,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1132,"Index":130,"Attempt":0,"Launch Time":1427397532395,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1129,"Index":127,"Attempt":0,"Launch Time":1427397532387,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532395,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":283612,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1133,"Index":131,"Attempt":0,"Launch Time":1427397532398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1130,"Index":128,"Attempt":0,"Launch Time":1427397532391,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532398,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":321045,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1134,"Index":132,"Attempt":0,"Launch Time":1427397532399,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1131,"Index":129,"Attempt":0,"Launch Time":1427397532392,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532399,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":310061,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1135,"Index":133,"Attempt":0,"Launch Time":1427397532402,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1132,"Index":130,"Attempt":0,"Launch Time":1427397532395,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532402,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":313938,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1136,"Index":134,"Attempt":0,"Launch Time":1427397532407,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1133,"Index":131,"Attempt":0,"Launch Time":1427397532398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532408,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":316250,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1137,"Index":135,"Attempt":0,"Launch Time":1427397532409,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1134,"Index":132,"Attempt":0,"Launch Time":1427397532399,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532410,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":347561,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1138,"Index":136,"Attempt":0,"Launch Time":1427397532410,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1135,"Index":133,"Attempt":0,"Launch Time":1427397532402,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532410,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":333995,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1139,"Index":137,"Attempt":0,"Launch Time":1427397532417,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1137,"Index":135,"Attempt":0,"Launch Time":1427397532409,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532417,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":313113,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1140,"Index":138,"Attempt":0,"Launch Time":1427397532417,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1138,"Index":136,"Attempt":0,"Launch Time":1427397532410,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532418,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":307317,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1141,"Index":139,"Attempt":0,"Launch Time":1427397532418,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1136,"Index":134,"Attempt":0,"Launch Time":1427397532407,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532418,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":310795,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1142,"Index":140,"Attempt":0,"Launch Time":1427397532425,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1139,"Index":137,"Attempt":0,"Launch Time":1427397532417,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532425,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":360993,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1143,"Index":141,"Attempt":0,"Launch Time":1427397532426,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1140,"Index":138,"Attempt":0,"Launch Time":1427397532417,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532426,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":388004,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1144,"Index":142,"Attempt":0,"Launch Time":1427397532429,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1141,"Index":139,"Attempt":0,"Launch Time":1427397532418,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532429,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":2594911,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1145,"Index":143,"Attempt":0,"Launch Time":1427397532434,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1143,"Index":141,"Attempt":0,"Launch Time":1427397532426,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532434,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":335554,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1146,"Index":144,"Attempt":0,"Launch Time":1427397532435,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1142,"Index":140,"Attempt":0,"Launch Time":1427397532425,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532435,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9606,"Shuffle Write Time":317795,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1147,"Index":145,"Attempt":0,"Launch Time":1427397532436,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1144,"Index":142,"Attempt":0,"Launch Time":1427397532429,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532436,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":322998,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1148,"Index":146,"Attempt":0,"Launch Time":1427397532462,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1103,"Index":101,"Attempt":0,"Launch Time":1427397532310,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532472,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":20,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":359286,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1149,"Index":147,"Attempt":0,"Launch Time":1427397532472,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1150,"Index":148,"Attempt":0,"Launch Time":1427397532472,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1107,"Index":105,"Attempt":0,"Launch Time":1427397532316,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532472,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":374846,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1145,"Index":143,"Attempt":0,"Launch Time":1427397532434,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532472,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":28,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":349836,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1151,"Index":149,"Attempt":0,"Launch Time":1427397532472,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1152,"Index":150,"Attempt":0,"Launch Time":1427397532473,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1101,"Index":99,"Attempt":0,"Launch Time":1427397532307,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532473,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":161,"Result Size":930,"JVM GC Time":14,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":310226,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1146,"Index":144,"Attempt":0,"Launch Time":1427397532435,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532473,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":36,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":289152,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1153,"Index":151,"Attempt":0,"Launch Time":1427397532473,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1105,"Index":103,"Attempt":0,"Launch Time":1427397532312,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532473,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":23,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":365869,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1154,"Index":152,"Attempt":0,"Launch Time":1427397532473,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1106,"Index":104,"Attempt":0,"Launch Time":1427397532314,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532473,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":22,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":738861,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1155,"Index":153,"Attempt":0,"Launch Time":1427397532483,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1151,"Index":149,"Attempt":0,"Launch Time":1427397532472,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532483,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":348594,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1156,"Index":154,"Attempt":0,"Launch Time":1427397532484,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1152,"Index":150,"Attempt":0,"Launch Time":1427397532473,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532484,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":374266,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1157,"Index":155,"Attempt":0,"Launch Time":1427397532484,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1150,"Index":148,"Attempt":0,"Launch Time":1427397532472,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532484,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":328534,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1158,"Index":156,"Attempt":0,"Launch Time":1427397532485,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1154,"Index":152,"Attempt":0,"Launch Time":1427397532473,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532485,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":353992,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1159,"Index":157,"Attempt":0,"Launch Time":1427397532489,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1149,"Index":147,"Attempt":0,"Launch Time":1427397532472,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532489,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":334310,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1160,"Index":158,"Attempt":0,"Launch Time":1427397532491,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1155,"Index":153,"Attempt":0,"Launch Time":1427397532483,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532491,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":352776,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1161,"Index":159,"Attempt":0,"Launch Time":1427397532492,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1156,"Index":154,"Attempt":0,"Launch Time":1427397532484,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532492,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":338051,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1162,"Index":160,"Attempt":0,"Launch Time":1427397532493,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1157,"Index":155,"Attempt":0,"Launch Time":1427397532484,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532493,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":315182,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1163,"Index":161,"Attempt":0,"Launch Time":1427397532495,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1148,"Index":146,"Attempt":0,"Launch Time":1427397532462,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532495,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":23,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":664614,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1164,"Index":162,"Attempt":0,"Launch Time":1427397532496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1158,"Index":156,"Attempt":0,"Launch Time":1427397532485,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532496,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1238925,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1165,"Index":163,"Attempt":0,"Launch Time":1427397532499,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1160,"Index":158,"Attempt":0,"Launch Time":1427397532491,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532499,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":327843,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1166,"Index":164,"Attempt":0,"Launch Time":1427397532499,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1159,"Index":157,"Attempt":0,"Launch Time":1427397532489,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532499,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":301485,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1167,"Index":165,"Attempt":0,"Launch Time":1427397532500,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1161,"Index":159,"Attempt":0,"Launch Time":1427397532492,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532501,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":263381,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1168,"Index":166,"Attempt":0,"Launch Time":1427397532502,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1163,"Index":161,"Attempt":0,"Launch Time":1427397532495,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532502,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":284390,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1169,"Index":167,"Attempt":0,"Launch Time":1427397532503,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1164,"Index":162,"Attempt":0,"Launch Time":1427397532496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532503,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":287640,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1170,"Index":168,"Attempt":0,"Launch Time":1427397532505,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1165,"Index":163,"Attempt":0,"Launch Time":1427397532499,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532506,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":265607,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1171,"Index":169,"Attempt":0,"Launch Time":1427397532508,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1166,"Index":164,"Attempt":0,"Launch Time":1427397532499,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532508,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":301599,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1172,"Index":170,"Attempt":0,"Launch Time":1427397532510,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1167,"Index":165,"Attempt":0,"Launch Time":1427397532500,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532510,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":340735,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1173,"Index":171,"Attempt":0,"Launch Time":1427397532511,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1168,"Index":166,"Attempt":0,"Launch Time":1427397532502,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532511,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":319746,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1174,"Index":172,"Attempt":0,"Launch Time":1427397532513,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1153,"Index":151,"Attempt":0,"Launch Time":1427397532473,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532513,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":40,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":8517822,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1170,"Index":168,"Attempt":0,"Launch Time":1427397532505,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532514,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":348247,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1175,"Index":173,"Attempt":0,"Launch Time":1427397532514,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1176,"Index":174,"Attempt":0,"Launch Time":1427397532514,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1162,"Index":160,"Attempt":0,"Launch Time":1427397532493,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532515,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":3122989,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1177,"Index":175,"Attempt":0,"Launch Time":1427397532517,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1178,"Index":176,"Attempt":0,"Launch Time":1427397532517,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1171,"Index":169,"Attempt":0,"Launch Time":1427397532508,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532517,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":452835,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1169,"Index":167,"Attempt":0,"Launch Time":1427397532503,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532517,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":437685,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1179,"Index":177,"Attempt":0,"Launch Time":1427397532518,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1172,"Index":170,"Attempt":0,"Launch Time":1427397532510,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532518,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":348229,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1180,"Index":178,"Attempt":0,"Launch Time":1427397532522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1176,"Index":174,"Attempt":0,"Launch Time":1427397532514,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532522,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":297324,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1181,"Index":179,"Attempt":0,"Launch Time":1427397532524,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1175,"Index":173,"Attempt":0,"Launch Time":1427397532514,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532524,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":321764,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1182,"Index":180,"Attempt":0,"Launch Time":1427397532525,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1183,"Index":181,"Attempt":0,"Launch Time":1427397532525,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1184,"Index":182,"Attempt":0,"Launch Time":1427397532525,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1177,"Index":175,"Attempt":0,"Launch Time":1427397532517,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532525,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":320698,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1178,"Index":176,"Attempt":0,"Launch Time":1427397532517,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532525,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":307507,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1174,"Index":172,"Attempt":0,"Launch Time":1427397532513,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532526,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":320886,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1185,"Index":183,"Attempt":0,"Launch Time":1427397532531,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1179,"Index":177,"Attempt":0,"Launch Time":1427397532518,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532531,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":446490,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1186,"Index":184,"Attempt":0,"Launch Time":1427397532532,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1180,"Index":178,"Attempt":0,"Launch Time":1427397532522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532532,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":341378,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1187,"Index":185,"Attempt":0,"Launch Time":1427397532533,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1184,"Index":182,"Attempt":0,"Launch Time":1427397532525,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532533,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":351714,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1188,"Index":186,"Attempt":0,"Launch Time":1427397532534,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1182,"Index":180,"Attempt":0,"Launch Time":1427397532525,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532534,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":331960,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1189,"Index":187,"Attempt":0,"Launch Time":1427397532535,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1181,"Index":179,"Attempt":0,"Launch Time":1427397532524,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532535,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":555430,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1190,"Index":188,"Attempt":0,"Launch Time":1427397532535,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1183,"Index":181,"Attempt":0,"Launch Time":1427397532525,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532535,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":323979,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1191,"Index":189,"Attempt":0,"Launch Time":1427397532546,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1187,"Index":185,"Attempt":0,"Launch Time":1427397532533,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532546,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":298482,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1192,"Index":190,"Attempt":0,"Launch Time":1427397532553,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1191,"Index":189,"Attempt":0,"Launch Time":1427397532546,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532553,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":324423,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1193,"Index":191,"Attempt":0,"Launch Time":1427397532561,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1192,"Index":190,"Attempt":0,"Launch Time":1427397532553,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532561,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":329400,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1194,"Index":192,"Attempt":0,"Launch Time":1427397532562,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1186,"Index":184,"Attempt":0,"Launch Time":1427397532532,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532562,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":28,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":328113,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1195,"Index":193,"Attempt":0,"Launch Time":1427397532562,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1188,"Index":186,"Attempt":0,"Launch Time":1427397532534,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532562,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":27,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":284054,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1196,"Index":194,"Attempt":0,"Launch Time":1427397532562,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1190,"Index":188,"Attempt":0,"Launch Time":1427397532535,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532563,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":27,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":316529,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1197,"Index":195,"Attempt":0,"Launch Time":1427397532563,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1173,"Index":171,"Attempt":0,"Launch Time":1427397532511,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532563,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":7,"Executor Run Time":44,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":19745151,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1198,"Index":196,"Attempt":0,"Launch Time":1427397532565,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1185,"Index":183,"Attempt":0,"Launch Time":1427397532531,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532565,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":32,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":335039,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1199,"Index":197,"Attempt":0,"Launch Time":1427397532566,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1189,"Index":187,"Attempt":0,"Launch Time":1427397532535,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532566,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":31,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":343253,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1200,"Index":198,"Attempt":0,"Launch Time":1427397532568,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1147,"Index":145,"Attempt":0,"Launch Time":1427397532436,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532568,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":16,"Executor Run Time":96,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":5252046,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1201,"Index":199,"Attempt":0,"Launch Time":1427397532571,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1197,"Index":195,"Attempt":0,"Launch Time":1427397532563,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532571,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":307748,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1202,"Index":200,"Attempt":0,"Launch Time":1427397532574,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1199,"Index":197,"Attempt":0,"Launch Time":1427397532566,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532574,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":364824,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1203,"Index":201,"Attempt":0,"Launch Time":1427397532574,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1198,"Index":196,"Attempt":0,"Launch Time":1427397532565,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532575,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":373453,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1204,"Index":202,"Attempt":0,"Launch Time":1427397532578,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1200,"Index":198,"Attempt":0,"Launch Time":1427397532568,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532579,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":394834,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1205,"Index":203,"Attempt":0,"Launch Time":1427397532580,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1201,"Index":199,"Attempt":0,"Launch Time":1427397532571,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532580,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":459168,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1206,"Index":204,"Attempt":0,"Launch Time":1427397532582,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1202,"Index":200,"Attempt":0,"Launch Time":1427397532574,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532582,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":393321,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1207,"Index":205,"Attempt":0,"Launch Time":1427397532587,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1203,"Index":201,"Attempt":0,"Launch Time":1427397532574,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532587,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":344698,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1208,"Index":206,"Attempt":0,"Launch Time":1427397532588,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1205,"Index":203,"Attempt":0,"Launch Time":1427397532580,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532588,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":313718,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1209,"Index":207,"Attempt":0,"Launch Time":1427397532590,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1206,"Index":204,"Attempt":0,"Launch Time":1427397532582,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532590,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":349340,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1210,"Index":208,"Attempt":0,"Launch Time":1427397532590,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1204,"Index":202,"Attempt":0,"Launch Time":1427397532578,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532590,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":322496,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1211,"Index":209,"Attempt":0,"Launch Time":1427397532596,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1208,"Index":206,"Attempt":0,"Launch Time":1427397532588,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532596,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":408585,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1212,"Index":210,"Attempt":0,"Launch Time":1427397532597,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1213,"Index":211,"Attempt":0,"Launch Time":1427397532597,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1209,"Index":207,"Attempt":0,"Launch Time":1427397532590,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532598,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":333468,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1207,"Index":205,"Attempt":0,"Launch Time":1427397532587,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532598,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":349845,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1214,"Index":212,"Attempt":0,"Launch Time":1427397532598,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1210,"Index":208,"Attempt":0,"Launch Time":1427397532590,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532598,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":310599,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1215,"Index":213,"Attempt":0,"Launch Time":1427397532605,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1211,"Index":209,"Attempt":0,"Launch Time":1427397532596,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532605,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":364257,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1216,"Index":214,"Attempt":0,"Launch Time":1427397532607,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1213,"Index":211,"Attempt":0,"Launch Time":1427397532597,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532607,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":344072,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1217,"Index":215,"Attempt":0,"Launch Time":1427397532607,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1214,"Index":212,"Attempt":0,"Launch Time":1427397532598,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532607,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":359494,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1218,"Index":216,"Attempt":0,"Launch Time":1427397532608,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1212,"Index":210,"Attempt":0,"Launch Time":1427397532597,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532608,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":564813,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1219,"Index":217,"Attempt":0,"Launch Time":1427397532612,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1194,"Index":192,"Attempt":0,"Launch Time":1427397532562,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532612,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":40,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":540404,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1220,"Index":218,"Attempt":0,"Launch Time":1427397532614,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1215,"Index":213,"Attempt":0,"Launch Time":1427397532605,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532614,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":369813,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1221,"Index":219,"Attempt":0,"Launch Time":1427397532617,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1216,"Index":214,"Attempt":0,"Launch Time":1427397532607,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532617,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":401448,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1222,"Index":220,"Attempt":0,"Launch Time":1427397532619,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1218,"Index":216,"Attempt":0,"Launch Time":1427397532608,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532619,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":350873,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1223,"Index":221,"Attempt":0,"Launch Time":1427397532620,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1219,"Index":217,"Attempt":0,"Launch Time":1427397532612,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532620,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":312558,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1224,"Index":222,"Attempt":0,"Launch Time":1427397532622,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1220,"Index":218,"Attempt":0,"Launch Time":1427397532614,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532622,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":337216,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1225,"Index":223,"Attempt":0,"Launch Time":1427397532625,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1221,"Index":219,"Attempt":0,"Launch Time":1427397532617,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532625,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":311149,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1226,"Index":224,"Attempt":0,"Launch Time":1427397532625,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1217,"Index":215,"Attempt":0,"Launch Time":1427397532607,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532626,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":531759,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1227,"Index":225,"Attempt":0,"Launch Time":1427397532626,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1222,"Index":220,"Attempt":0,"Launch Time":1427397532619,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532627,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":316374,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1228,"Index":226,"Attempt":0,"Launch Time":1427397532627,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1223,"Index":221,"Attempt":0,"Launch Time":1427397532620,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532627,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":316416,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1229,"Index":227,"Attempt":0,"Launch Time":1427397532631,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1224,"Index":222,"Attempt":0,"Launch Time":1427397532622,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532631,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":341870,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1230,"Index":228,"Attempt":0,"Launch Time":1427397532633,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1225,"Index":223,"Attempt":0,"Launch Time":1427397532625,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532633,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":328956,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1231,"Index":229,"Attempt":0,"Launch Time":1427397532634,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1227,"Index":225,"Attempt":0,"Launch Time":1427397532626,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532634,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":306271,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1232,"Index":230,"Attempt":0,"Launch Time":1427397532648,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1229,"Index":227,"Attempt":0,"Launch Time":1427397532631,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532648,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":8,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":314846,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1233,"Index":231,"Attempt":0,"Launch Time":1427397532651,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1228,"Index":226,"Attempt":0,"Launch Time":1427397532627,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532651,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":24,"Result Size":930,"JVM GC Time":8,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":611848,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1234,"Index":232,"Attempt":0,"Launch Time":1427397532653,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1226,"Index":224,"Attempt":0,"Launch Time":1427397532625,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532653,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":27,"Result Size":930,"JVM GC Time":8,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":12767861,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1235,"Index":233,"Attempt":0,"Launch Time":1427397532655,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1230,"Index":228,"Attempt":0,"Launch Time":1427397532633,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532656,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":21,"Result Size":930,"JVM GC Time":8,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":618579,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1231,"Index":229,"Attempt":0,"Launch Time":1427397532634,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532656,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":20,"Result Size":930,"JVM GC Time":8,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":374977,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1236,"Index":234,"Attempt":0,"Launch Time":1427397532656,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1237,"Index":235,"Attempt":0,"Launch Time":1427397532656,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1232,"Index":230,"Attempt":0,"Launch Time":1427397532648,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532657,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":412688,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1238,"Index":236,"Attempt":0,"Launch Time":1427397532660,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1233,"Index":231,"Attempt":0,"Launch Time":1427397532651,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532660,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":466555,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1239,"Index":237,"Attempt":0,"Launch Time":1427397532664,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1240,"Index":238,"Attempt":0,"Launch Time":1427397532664,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1235,"Index":233,"Attempt":0,"Launch Time":1427397532655,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532664,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":362677,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1196,"Index":194,"Attempt":0,"Launch Time":1427397532562,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532664,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":36,"Result Size":930,"JVM GC Time":8,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":591905,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1241,"Index":239,"Attempt":0,"Launch Time":1427397532664,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1236,"Index":234,"Attempt":0,"Launch Time":1427397532656,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532665,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":372544,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1242,"Index":240,"Attempt":0,"Launch Time":1427397532665,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1234,"Index":232,"Attempt":0,"Launch Time":1427397532653,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532665,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":434915,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1243,"Index":241,"Attempt":0,"Launch Time":1427397532665,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1237,"Index":235,"Attempt":0,"Launch Time":1427397532656,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532665,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":365995,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1244,"Index":242,"Attempt":0,"Launch Time":1427397532667,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1195,"Index":193,"Attempt":0,"Launch Time":1427397532562,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532667,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":19,"Executor Run Time":39,"Result Size":930,"JVM GC Time":8,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":1816125,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1245,"Index":243,"Attempt":0,"Launch Time":1427397532672,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1239,"Index":237,"Attempt":0,"Launch Time":1427397532664,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532672,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":314249,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1246,"Index":244,"Attempt":0,"Launch Time":1427397532673,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1241,"Index":239,"Attempt":0,"Launch Time":1427397532664,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532673,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":309591,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1247,"Index":245,"Attempt":0,"Launch Time":1427397532673,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1248,"Index":246,"Attempt":0,"Launch Time":1427397532673,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1240,"Index":238,"Attempt":0,"Launch Time":1427397532664,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532673,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":317237,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1243,"Index":241,"Attempt":0,"Launch Time":1427397532665,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532674,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":293815,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1249,"Index":247,"Attempt":0,"Launch Time":1427397532675,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1238,"Index":236,"Attempt":0,"Launch Time":1427397532660,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532675,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":335638,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1250,"Index":248,"Attempt":0,"Launch Time":1427397532678,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1242,"Index":240,"Attempt":0,"Launch Time":1427397532665,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532678,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":353032,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1251,"Index":249,"Attempt":0,"Launch Time":1427397532680,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1252,"Index":250,"Attempt":0,"Launch Time":1427397532680,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1253,"Index":251,"Attempt":0,"Launch Time":1427397532681,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1247,"Index":245,"Attempt":0,"Launch Time":1427397532673,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532681,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":337849,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1246,"Index":244,"Attempt":0,"Launch Time":1427397532673,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532681,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":346204,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1248,"Index":246,"Attempt":0,"Launch Time":1427397532673,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532681,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":319346,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1254,"Index":252,"Attempt":0,"Launch Time":1427397532687,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1250,"Index":248,"Attempt":0,"Launch Time":1427397532678,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532687,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":331044,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1255,"Index":253,"Attempt":0,"Launch Time":1427397532693,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1244,"Index":242,"Attempt":0,"Launch Time":1427397532667,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532693,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":11,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":327859,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1256,"Index":254,"Attempt":0,"Launch Time":1427397532694,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1193,"Index":191,"Attempt":0,"Launch Time":1427397532561,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532694,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":39,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":294152,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1257,"Index":255,"Attempt":0,"Launch Time":1427397532696,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1245,"Index":243,"Attempt":0,"Launch Time":1427397532672,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532696,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":335077,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1258,"Index":256,"Attempt":0,"Launch Time":1427397532701,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1256,"Index":254,"Attempt":0,"Launch Time":1427397532694,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532701,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":344258,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1259,"Index":257,"Attempt":0,"Launch Time":1427397532702,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1255,"Index":253,"Attempt":0,"Launch Time":1427397532693,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532702,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":340481,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1260,"Index":258,"Attempt":0,"Launch Time":1427397532704,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1257,"Index":255,"Attempt":0,"Launch Time":1427397532696,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532705,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":340585,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1261,"Index":259,"Attempt":0,"Launch Time":1427397532710,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1262,"Index":260,"Attempt":0,"Launch Time":1427397532711,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1258,"Index":256,"Attempt":0,"Launch Time":1427397532701,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532711,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":386718,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1259,"Index":257,"Attempt":0,"Launch Time":1427397532702,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532711,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9613,"Shuffle Write Time":393368,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1263,"Index":261,"Attempt":0,"Launch Time":1427397532713,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1260,"Index":258,"Attempt":0,"Launch Time":1427397532704,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532713,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":465471,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1264,"Index":262,"Attempt":0,"Launch Time":1427397532719,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1265,"Index":263,"Attempt":0,"Launch Time":1427397532719,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1262,"Index":260,"Attempt":0,"Launch Time":1427397532711,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532722,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":354936,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1249,"Index":247,"Attempt":0,"Launch Time":1427397532675,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532722,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":42,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":2135654,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1266,"Index":264,"Attempt":0,"Launch Time":1427397532726,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1265,"Index":263,"Attempt":0,"Launch Time":1427397532719,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532726,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":327535,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1267,"Index":265,"Attempt":0,"Launch Time":1427397532726,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1264,"Index":262,"Attempt":0,"Launch Time":1427397532719,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532726,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":323379,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1268,"Index":266,"Attempt":0,"Launch Time":1427397532729,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1263,"Index":261,"Attempt":0,"Launch Time":1427397532713,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532729,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":337801,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1269,"Index":267,"Attempt":0,"Launch Time":1427397532730,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1261,"Index":259,"Attempt":0,"Launch Time":1427397532710,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532730,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":432272,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1270,"Index":268,"Attempt":0,"Launch Time":1427397532733,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1266,"Index":264,"Attempt":0,"Launch Time":1427397532726,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532734,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":314134,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1271,"Index":269,"Attempt":0,"Launch Time":1427397532736,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1268,"Index":266,"Attempt":0,"Launch Time":1427397532729,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532736,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":328701,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1272,"Index":270,"Attempt":0,"Launch Time":1427397532737,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1269,"Index":267,"Attempt":0,"Launch Time":1427397532730,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532737,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":319713,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1273,"Index":271,"Attempt":0,"Launch Time":1427397532767,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1270,"Index":268,"Attempt":0,"Launch Time":1427397532733,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532767,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":33,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":327160,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1274,"Index":272,"Attempt":0,"Launch Time":1427397532769,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1267,"Index":265,"Attempt":0,"Launch Time":1427397532726,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532769,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":41,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9613,"Shuffle Write Time":487730,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1275,"Index":273,"Attempt":0,"Launch Time":1427397532774,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1251,"Index":249,"Attempt":0,"Launch Time":1427397532680,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532774,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":93,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9613,"Shuffle Write Time":375765,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1276,"Index":274,"Attempt":0,"Launch Time":1427397532775,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1271,"Index":269,"Attempt":0,"Launch Time":1427397532736,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532775,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":38,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":392591,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1277,"Index":275,"Attempt":0,"Launch Time":1427397532775,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1273,"Index":271,"Attempt":0,"Launch Time":1427397532767,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532775,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":361830,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1278,"Index":276,"Attempt":0,"Launch Time":1427397532777,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1274,"Index":272,"Attempt":0,"Launch Time":1427397532769,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532777,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":349932,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1279,"Index":277,"Attempt":0,"Launch Time":1427397532779,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1272,"Index":270,"Attempt":0,"Launch Time":1427397532737,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532779,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":41,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":331741,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1280,"Index":278,"Attempt":0,"Launch Time":1427397532783,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1276,"Index":274,"Attempt":0,"Launch Time":1427397532775,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532783,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":350696,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1277,"Index":275,"Attempt":0,"Launch Time":1427397532775,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532783,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":363488,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1281,"Index":279,"Attempt":0,"Launch Time":1427397532783,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1282,"Index":280,"Attempt":0,"Launch Time":1427397532786,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1279,"Index":277,"Attempt":0,"Launch Time":1427397532779,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532786,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":278799,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1283,"Index":281,"Attempt":0,"Launch Time":1427397532791,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1281,"Index":279,"Attempt":0,"Launch Time":1427397532783,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532792,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":353414,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1284,"Index":282,"Attempt":0,"Launch Time":1427397532792,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1280,"Index":278,"Attempt":0,"Launch Time":1427397532783,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532792,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":417842,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1285,"Index":283,"Attempt":0,"Launch Time":1427397532794,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1286,"Index":284,"Attempt":0,"Launch Time":1427397532794,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1278,"Index":276,"Attempt":0,"Launch Time":1427397532777,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532794,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":917389,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1282,"Index":280,"Attempt":0,"Launch Time":1427397532786,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532795,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":370383,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1287,"Index":285,"Attempt":0,"Launch Time":1427397532795,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1254,"Index":252,"Attempt":0,"Launch Time":1427397532687,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532795,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":53,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":854377,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1288,"Index":286,"Attempt":0,"Launch Time":1427397532798,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1283,"Index":281,"Attempt":0,"Launch Time":1427397532791,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532799,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":309781,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1289,"Index":287,"Attempt":0,"Launch Time":1427397532800,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1284,"Index":282,"Attempt":0,"Launch Time":1427397532792,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532800,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":317403,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1290,"Index":288,"Attempt":0,"Launch Time":1427397532802,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1285,"Index":283,"Attempt":0,"Launch Time":1427397532794,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532802,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":317079,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1291,"Index":289,"Attempt":0,"Launch Time":1427397532803,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1286,"Index":284,"Attempt":0,"Launch Time":1427397532794,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532803,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":326418,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1292,"Index":290,"Attempt":0,"Launch Time":1427397532805,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1288,"Index":286,"Attempt":0,"Launch Time":1427397532798,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532805,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":315472,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1293,"Index":291,"Attempt":0,"Launch Time":1427397532809,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1252,"Index":250,"Attempt":0,"Launch Time":1427397532680,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532810,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":128,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":377423,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1294,"Index":292,"Attempt":0,"Launch Time":1427397532810,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1289,"Index":287,"Attempt":0,"Launch Time":1427397532800,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532811,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":407204,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1295,"Index":293,"Attempt":0,"Launch Time":1427397532812,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1253,"Index":251,"Attempt":0,"Launch Time":1427397532681,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532812,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":131,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":366540,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1296,"Index":294,"Attempt":0,"Launch Time":1427397532813,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1290,"Index":288,"Attempt":0,"Launch Time":1427397532802,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532813,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":351599,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1297,"Index":295,"Attempt":0,"Launch Time":1427397532815,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1275,"Index":273,"Attempt":0,"Launch Time":1427397532774,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532815,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":40,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":420051,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1298,"Index":296,"Attempt":0,"Launch Time":1427397532818,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1293,"Index":291,"Attempt":0,"Launch Time":1427397532809,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532818,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":329158,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1299,"Index":297,"Attempt":0,"Launch Time":1427397532819,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1287,"Index":285,"Attempt":0,"Launch Time":1427397532795,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532819,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":23,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":331023,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1300,"Index":298,"Attempt":0,"Launch Time":1427397532822,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1301,"Index":299,"Attempt":0,"Launch Time":1427397532822,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1297,"Index":295,"Attempt":0,"Launch Time":1427397532815,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532822,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":327543,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1295,"Index":293,"Attempt":0,"Launch Time":1427397532812,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532822,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":328166,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1302,"Index":300,"Attempt":0,"Launch Time":1427397532825,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1298,"Index":296,"Attempt":0,"Launch Time":1427397532818,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532826,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":318832,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1303,"Index":301,"Attempt":0,"Launch Time":1427397532832,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1300,"Index":298,"Attempt":0,"Launch Time":1427397532822,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532832,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":336423,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1304,"Index":302,"Attempt":0,"Launch Time":1427397532833,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1302,"Index":300,"Attempt":0,"Launch Time":1427397532825,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532833,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":324674,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1305,"Index":303,"Attempt":0,"Launch Time":1427397532835,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1301,"Index":299,"Attempt":0,"Launch Time":1427397532822,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532836,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":302159,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1306,"Index":304,"Attempt":0,"Launch Time":1427397532840,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1307,"Index":305,"Attempt":0,"Launch Time":1427397532840,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1304,"Index":302,"Attempt":0,"Launch Time":1427397532833,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532840,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":317738,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1303,"Index":301,"Attempt":0,"Launch Time":1427397532832,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532840,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":347409,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1308,"Index":306,"Attempt":0,"Launch Time":1427397532846,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1305,"Index":303,"Attempt":0,"Launch Time":1427397532835,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532847,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":400863,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1309,"Index":307,"Attempt":0,"Launch Time":1427397532851,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1299,"Index":297,"Attempt":0,"Launch Time":1427397532819,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532851,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":30,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":2422447,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1310,"Index":308,"Attempt":0,"Launch Time":1427397532851,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1306,"Index":304,"Attempt":0,"Launch Time":1427397532840,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532851,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":363044,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1311,"Index":309,"Attempt":0,"Launch Time":1427397532852,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1307,"Index":305,"Attempt":0,"Launch Time":1427397532840,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532852,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":352671,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1312,"Index":310,"Attempt":0,"Launch Time":1427397532854,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1308,"Index":306,"Attempt":0,"Launch Time":1427397532846,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532854,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":312259,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1313,"Index":311,"Attempt":0,"Launch Time":1427397532861,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1310,"Index":308,"Attempt":0,"Launch Time":1427397532851,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532861,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":339970,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1314,"Index":312,"Attempt":0,"Launch Time":1427397532862,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1309,"Index":307,"Attempt":0,"Launch Time":1427397532851,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532862,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":329037,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1315,"Index":313,"Attempt":0,"Launch Time":1427397532863,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1311,"Index":309,"Attempt":0,"Launch Time":1427397532852,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532863,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":386906,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1316,"Index":314,"Attempt":0,"Launch Time":1427397532864,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1296,"Index":294,"Attempt":0,"Launch Time":1427397532813,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532864,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":377329,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1317,"Index":315,"Attempt":0,"Launch Time":1427397532879,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1314,"Index":312,"Attempt":0,"Launch Time":1427397532862,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532879,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":351936,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1318,"Index":316,"Attempt":0,"Launch Time":1427397532880,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1312,"Index":310,"Attempt":0,"Launch Time":1427397532854,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532880,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":26,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":705379,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1319,"Index":317,"Attempt":0,"Launch Time":1427397532882,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1313,"Index":311,"Attempt":0,"Launch Time":1427397532861,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532883,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":326653,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1320,"Index":318,"Attempt":0,"Launch Time":1427397532891,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1319,"Index":317,"Attempt":0,"Launch Time":1427397532882,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532891,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":415240,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1318,"Index":316,"Attempt":0,"Launch Time":1427397532880,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532891,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":443144,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1321,"Index":319,"Attempt":0,"Launch Time":1427397532891,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1322,"Index":320,"Attempt":0,"Launch Time":1427397532891,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1316,"Index":314,"Attempt":0,"Launch Time":1427397532864,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532892,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":27,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":439879,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1323,"Index":321,"Attempt":0,"Launch Time":1427397532894,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1315,"Index":313,"Attempt":0,"Launch Time":1427397532863,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532894,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":29,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":526743,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1324,"Index":322,"Attempt":0,"Launch Time":1427397532898,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1317,"Index":315,"Attempt":0,"Launch Time":1427397532879,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532899,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":491754,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1325,"Index":323,"Attempt":0,"Launch Time":1427397532900,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1322,"Index":320,"Attempt":0,"Launch Time":1427397532891,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532900,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":376957,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1326,"Index":324,"Attempt":0,"Launch Time":1427397532900,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1320,"Index":318,"Attempt":0,"Launch Time":1427397532891,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532900,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":370996,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1327,"Index":325,"Attempt":0,"Launch Time":1427397532901,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1321,"Index":319,"Attempt":0,"Launch Time":1427397532891,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532901,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":435376,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1328,"Index":326,"Attempt":0,"Launch Time":1427397532903,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1323,"Index":321,"Attempt":0,"Launch Time":1427397532894,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532904,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":513472,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1329,"Index":327,"Attempt":0,"Launch Time":1427397532904,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1291,"Index":289,"Attempt":0,"Launch Time":1427397532803,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532904,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":100,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":359404,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1330,"Index":328,"Attempt":0,"Launch Time":1427397532908,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1326,"Index":324,"Attempt":0,"Launch Time":1427397532900,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532908,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":334586,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1331,"Index":329,"Attempt":0,"Launch Time":1427397532909,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1325,"Index":323,"Attempt":0,"Launch Time":1427397532900,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532910,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":604949,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1332,"Index":330,"Attempt":0,"Launch Time":1427397532910,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1327,"Index":325,"Attempt":0,"Launch Time":1427397532901,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532910,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":323699,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1333,"Index":331,"Attempt":0,"Launch Time":1427397532912,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1328,"Index":326,"Attempt":0,"Launch Time":1427397532903,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532912,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":356945,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1334,"Index":332,"Attempt":0,"Launch Time":1427397532915,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1330,"Index":328,"Attempt":0,"Launch Time":1427397532908,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532915,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":329104,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1335,"Index":333,"Attempt":0,"Launch Time":1427397532916,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1292,"Index":290,"Attempt":0,"Launch Time":1427397532805,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532916,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":110,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":379183,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1336,"Index":334,"Attempt":0,"Launch Time":1427397532917,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1331,"Index":329,"Attempt":0,"Launch Time":1427397532909,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532918,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":348073,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1337,"Index":335,"Attempt":0,"Launch Time":1427397532920,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1333,"Index":331,"Attempt":0,"Launch Time":1427397532912,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532920,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":369575,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1338,"Index":336,"Attempt":0,"Launch Time":1427397532924,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1332,"Index":330,"Attempt":0,"Launch Time":1427397532910,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532924,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":353868,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1339,"Index":337,"Attempt":0,"Launch Time":1427397532924,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1335,"Index":333,"Attempt":0,"Launch Time":1427397532916,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532924,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":339069,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1340,"Index":338,"Attempt":0,"Launch Time":1427397532928,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1336,"Index":334,"Attempt":0,"Launch Time":1427397532917,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532928,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9666,"Shuffle Write Time":349449,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1341,"Index":339,"Attempt":0,"Launch Time":1427397532929,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1337,"Index":335,"Attempt":0,"Launch Time":1427397532920,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532929,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9650,"Shuffle Write Time":337146,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1342,"Index":340,"Attempt":0,"Launch Time":1427397532930,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1338,"Index":336,"Attempt":0,"Launch Time":1427397532924,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532931,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9648,"Shuffle Write Time":314161,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1343,"Index":341,"Attempt":0,"Launch Time":1427397532931,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1339,"Index":337,"Attempt":0,"Launch Time":1427397532924,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532931,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9651,"Shuffle Write Time":312281,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1344,"Index":342,"Attempt":0,"Launch Time":1427397532935,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1340,"Index":338,"Attempt":0,"Launch Time":1427397532928,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532935,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9651,"Shuffle Write Time":337520,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1345,"Index":343,"Attempt":0,"Launch Time":1427397532936,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1329,"Index":327,"Attempt":0,"Launch Time":1427397532904,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532936,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":361406,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1346,"Index":344,"Attempt":0,"Launch Time":1427397532940,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1294,"Index":292,"Attempt":0,"Launch Time":1427397532810,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532940,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":62,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":405322,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1347,"Index":345,"Attempt":0,"Launch Time":1427397532942,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1334,"Index":332,"Attempt":0,"Launch Time":1427397532915,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532942,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":20,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":380556,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1348,"Index":346,"Attempt":0,"Launch Time":1427397532942,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1341,"Index":339,"Attempt":0,"Launch Time":1427397532929,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532943,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9647,"Shuffle Write Time":389321,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1349,"Index":347,"Attempt":0,"Launch Time":1427397532948,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1346,"Index":344,"Attempt":0,"Launch Time":1427397532940,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532949,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":393492,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1350,"Index":348,"Attempt":0,"Launch Time":1427397532952,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1345,"Index":343,"Attempt":0,"Launch Time":1427397532936,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532953,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":346476,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1351,"Index":349,"Attempt":0,"Launch Time":1427397532956,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1348,"Index":346,"Attempt":0,"Launch Time":1427397532942,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532957,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":329862,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1352,"Index":350,"Attempt":0,"Launch Time":1427397532957,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1349,"Index":347,"Attempt":0,"Launch Time":1427397532948,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532957,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":334898,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1353,"Index":351,"Attempt":0,"Launch Time":1427397532964,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1351,"Index":349,"Attempt":0,"Launch Time":1427397532956,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532964,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":361912,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1354,"Index":352,"Attempt":0,"Launch Time":1427397532965,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1350,"Index":348,"Attempt":0,"Launch Time":1427397532952,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532965,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":372746,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1344,"Index":342,"Attempt":0,"Launch Time":1427397532935,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532966,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":31,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":375342,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1355,"Index":353,"Attempt":0,"Launch Time":1427397532967,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1356,"Index":354,"Attempt":0,"Launch Time":1427397532970,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1352,"Index":350,"Attempt":0,"Launch Time":1427397532957,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532970,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":330282,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1357,"Index":355,"Attempt":0,"Launch Time":1427397532972,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1354,"Index":352,"Attempt":0,"Launch Time":1427397532965,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532972,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":336529,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1358,"Index":356,"Attempt":0,"Launch Time":1427397532973,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1353,"Index":351,"Attempt":0,"Launch Time":1427397532964,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532973,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":351254,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1359,"Index":357,"Attempt":0,"Launch Time":1427397532973,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1355,"Index":353,"Attempt":0,"Launch Time":1427397532967,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532973,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":293604,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1360,"Index":358,"Attempt":0,"Launch Time":1427397532986,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1359,"Index":357,"Attempt":0,"Launch Time":1427397532973,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532987,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":299740,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1361,"Index":359,"Attempt":0,"Launch Time":1427397532999,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1347,"Index":345,"Attempt":0,"Launch Time":1427397532942,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532999,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":338751,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1362,"Index":360,"Attempt":0,"Launch Time":1427397533002,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1343,"Index":341,"Attempt":0,"Launch Time":1427397532931,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533003,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":70,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":55125458,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1363,"Index":361,"Attempt":0,"Launch Time":1427397533005,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1342,"Index":340,"Attempt":0,"Launch Time":1427397532930,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533005,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":74,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9649,"Shuffle Write Time":67252055,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1364,"Index":362,"Attempt":0,"Launch Time":1427397533008,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1358,"Index":356,"Attempt":0,"Launch Time":1427397532973,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533009,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":35,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":324013,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1365,"Index":363,"Attempt":0,"Launch Time":1427397533010,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1357,"Index":355,"Attempt":0,"Launch Time":1427397532972,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533010,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":38,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":341063,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1366,"Index":364,"Attempt":0,"Launch Time":1427397533012,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1363,"Index":361,"Attempt":0,"Launch Time":1427397533005,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533013,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":322208,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1367,"Index":365,"Attempt":0,"Launch Time":1427397533013,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1356,"Index":354,"Attempt":0,"Launch Time":1427397532970,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533013,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":382433,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1368,"Index":366,"Attempt":0,"Launch Time":1427397533016,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1324,"Index":322,"Attempt":0,"Launch Time":1427397532898,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533016,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":116,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":2076105,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1369,"Index":367,"Attempt":0,"Launch Time":1427397533018,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1365,"Index":363,"Attempt":0,"Launch Time":1427397533010,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533018,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":321011,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1370,"Index":368,"Attempt":0,"Launch Time":1427397533021,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1366,"Index":364,"Attempt":0,"Launch Time":1427397533012,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533021,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":321871,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1371,"Index":369,"Attempt":0,"Launch Time":1427397533021,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1367,"Index":365,"Attempt":0,"Launch Time":1427397533013,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533021,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":374851,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1372,"Index":370,"Attempt":0,"Launch Time":1427397533023,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1368,"Index":366,"Attempt":0,"Launch Time":1427397533016,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533024,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":351019,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1373,"Index":371,"Attempt":0,"Launch Time":1427397533028,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1370,"Index":368,"Attempt":0,"Launch Time":1427397533021,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533028,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":311359,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1374,"Index":372,"Attempt":0,"Launch Time":1427397533031,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1364,"Index":362,"Attempt":0,"Launch Time":1427397533008,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533031,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":22,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":353423,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1375,"Index":373,"Attempt":0,"Launch Time":1427397533038,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1369,"Index":367,"Attempt":0,"Launch Time":1427397533018,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533038,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":348413,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1376,"Index":374,"Attempt":0,"Launch Time":1427397533039,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1371,"Index":369,"Attempt":0,"Launch Time":1427397533021,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533039,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":452377,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1377,"Index":375,"Attempt":0,"Launch Time":1427397533039,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1372,"Index":370,"Attempt":0,"Launch Time":1427397533023,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533039,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":5054761,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1378,"Index":376,"Attempt":0,"Launch Time":1427397533039,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1374,"Index":372,"Attempt":0,"Launch Time":1427397533031,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533039,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":342103,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1379,"Index":377,"Attempt":0,"Launch Time":1427397533058,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1376,"Index":374,"Attempt":0,"Launch Time":1427397533039,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533058,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9612,"Shuffle Write Time":355781,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1380,"Index":378,"Attempt":0,"Launch Time":1427397533058,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1381,"Index":379,"Attempt":0,"Launch Time":1427397533059,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1382,"Index":380,"Attempt":0,"Launch Time":1427397533060,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1383,"Index":381,"Attempt":0,"Launch Time":1427397533060,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1384,"Index":382,"Attempt":0,"Launch Time":1427397533060,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1385,"Index":383,"Attempt":0,"Launch Time":1427397533060,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1386,"Index":384,"Attempt":0,"Launch Time":1427397533060,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1377,"Index":375,"Attempt":0,"Launch Time":1427397533039,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533060,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":338775,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1362,"Index":360,"Attempt":0,"Launch Time":1427397533002,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533061,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":329876,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1361,"Index":359,"Attempt":0,"Launch Time":1427397532999,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533061,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":646712,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1373,"Index":371,"Attempt":0,"Launch Time":1427397533028,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533061,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":817743,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1378,"Index":376,"Attempt":0,"Launch Time":1427397533039,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533061,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":414813,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1375,"Index":373,"Attempt":0,"Launch Time":1427397533038,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533061,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":348309,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1360,"Index":358,"Attempt":0,"Launch Time":1427397532986,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533061,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":49,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":375216,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1387,"Index":385,"Attempt":0,"Launch Time":1427397533065,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1379,"Index":377,"Attempt":0,"Launch Time":1427397533058,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533065,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":326027,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1388,"Index":386,"Attempt":0,"Launch Time":1427397533069,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1381,"Index":379,"Attempt":0,"Launch Time":1427397533059,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533070,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":342021,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1389,"Index":387,"Attempt":0,"Launch Time":1427397533075,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1386,"Index":384,"Attempt":0,"Launch Time":1427397533060,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533075,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":365336,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1390,"Index":388,"Attempt":0,"Launch Time":1427397533082,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1388,"Index":386,"Attempt":0,"Launch Time":1427397533069,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533082,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":644995,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1391,"Index":389,"Attempt":0,"Launch Time":1427397533084,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1389,"Index":387,"Attempt":0,"Launch Time":1427397533075,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533084,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":334979,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1392,"Index":390,"Attempt":0,"Launch Time":1427397533089,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1382,"Index":380,"Attempt":0,"Launch Time":1427397533060,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533090,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":24,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":353101,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1393,"Index":391,"Attempt":0,"Launch Time":1427397533090,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1390,"Index":388,"Attempt":0,"Launch Time":1427397533082,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533090,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":349927,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1394,"Index":392,"Attempt":0,"Launch Time":1427397533092,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1391,"Index":389,"Attempt":0,"Launch Time":1427397533084,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533092,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":346948,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1395,"Index":393,"Attempt":0,"Launch Time":1427397533092,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1384,"Index":382,"Attempt":0,"Launch Time":1427397533060,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533092,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":27,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9614,"Shuffle Write Time":816812,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1396,"Index":394,"Attempt":0,"Launch Time":1427397533098,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1393,"Index":391,"Attempt":0,"Launch Time":1427397533090,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533098,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":378503,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1397,"Index":395,"Attempt":0,"Launch Time":1427397533098,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1392,"Index":390,"Attempt":0,"Launch Time":1427397533089,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533098,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9615,"Shuffle Write Time":363196,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1398,"Index":396,"Attempt":0,"Launch Time":1427397533100,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1394,"Index":392,"Attempt":0,"Launch Time":1427397533092,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533100,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":392016,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1399,"Index":397,"Attempt":0,"Launch Time":1427397533104,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1395,"Index":393,"Attempt":0,"Launch Time":1427397533092,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533104,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":455851,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1400,"Index":398,"Attempt":0,"Launch Time":1427397533106,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1397,"Index":395,"Attempt":0,"Launch Time":1427397533098,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533107,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":384131,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1401,"Index":399,"Attempt":0,"Launch Time":1427397533107,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1402,"Index":400,"Attempt":0,"Launch Time":1427397533107,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1398,"Index":396,"Attempt":0,"Launch Time":1427397533100,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533107,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":321724,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1396,"Index":394,"Attempt":0,"Launch Time":1427397533098,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533107,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":331657,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1403,"Index":401,"Attempt":0,"Launch Time":1427397533121,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1402,"Index":400,"Attempt":0,"Launch Time":1427397533107,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533121,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":312660,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1404,"Index":402,"Attempt":0,"Launch Time":1427397533122,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1399,"Index":397,"Attempt":0,"Launch Time":1427397533104,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533122,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":14,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":542014,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1405,"Index":403,"Attempt":0,"Launch Time":1427397533130,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1406,"Index":404,"Attempt":0,"Launch Time":1427397533130,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1380,"Index":378,"Attempt":0,"Launch Time":1427397533058,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533130,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":36,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":404351,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1403,"Index":401,"Attempt":0,"Launch Time":1427397533121,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533130,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":398458,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1407,"Index":405,"Attempt":0,"Launch Time":1427397533131,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1404,"Index":402,"Attempt":0,"Launch Time":1427397533122,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533131,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":388487,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1408,"Index":406,"Attempt":0,"Launch Time":1427397533137,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1405,"Index":403,"Attempt":0,"Launch Time":1427397533130,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533137,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":338535,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1409,"Index":407,"Attempt":0,"Launch Time":1427397533138,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1407,"Index":405,"Attempt":0,"Launch Time":1427397533131,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533138,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":362557,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1410,"Index":408,"Attempt":0,"Launch Time":1427397533139,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1406,"Index":404,"Attempt":0,"Launch Time":1427397533130,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533139,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":318738,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1411,"Index":409,"Attempt":0,"Launch Time":1427397533146,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1409,"Index":407,"Attempt":0,"Launch Time":1427397533138,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533146,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":400622,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1412,"Index":410,"Attempt":0,"Launch Time":1427397533147,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1410,"Index":408,"Attempt":0,"Launch Time":1427397533139,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533148,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":435193,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1413,"Index":411,"Attempt":0,"Launch Time":1427397533156,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1412,"Index":410,"Attempt":0,"Launch Time":1427397533147,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533156,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":362480,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1414,"Index":412,"Attempt":0,"Launch Time":1427397533158,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1411,"Index":409,"Attempt":0,"Launch Time":1427397533146,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533158,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":359313,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1415,"Index":413,"Attempt":0,"Launch Time":1427397533159,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1408,"Index":406,"Attempt":0,"Launch Time":1427397533137,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533159,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":22,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":915274,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1416,"Index":414,"Attempt":0,"Launch Time":1427397533164,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1413,"Index":411,"Attempt":0,"Launch Time":1427397533156,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533164,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":375015,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1417,"Index":415,"Attempt":0,"Launch Time":1427397533166,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1414,"Index":412,"Attempt":0,"Launch Time":1427397533158,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533166,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":361990,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1418,"Index":416,"Attempt":0,"Launch Time":1427397533169,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1383,"Index":381,"Attempt":0,"Launch Time":1427397533060,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533169,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":108,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":3208002,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1419,"Index":417,"Attempt":0,"Launch Time":1427397533173,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1416,"Index":414,"Attempt":0,"Launch Time":1427397533164,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533173,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":368790,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1420,"Index":418,"Attempt":0,"Launch Time":1427397533178,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1387,"Index":385,"Attempt":0,"Launch Time":1427397533065,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533178,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":112,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":17534191,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1421,"Index":419,"Attempt":0,"Launch Time":1427397533178,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1400,"Index":398,"Attempt":0,"Launch Time":1427397533106,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533178,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":71,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":446961,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1422,"Index":420,"Attempt":0,"Launch Time":1427397533179,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1401,"Index":399,"Attempt":0,"Launch Time":1427397533107,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533180,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":71,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":390159,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1423,"Index":421,"Attempt":0,"Launch Time":1427397533184,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1415,"Index":413,"Attempt":0,"Launch Time":1427397533159,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533184,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":347534,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1424,"Index":422,"Attempt":0,"Launch Time":1427397533186,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1421,"Index":419,"Attempt":0,"Launch Time":1427397533178,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533186,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":331963,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1425,"Index":423,"Attempt":0,"Launch Time":1427397533188,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1426,"Index":424,"Attempt":0,"Launch Time":1427397533188,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1422,"Index":420,"Attempt":0,"Launch Time":1427397533179,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533188,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":339698,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1385,"Index":383,"Attempt":0,"Launch Time":1427397533060,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533188,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":19,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":364829,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1427,"Index":425,"Attempt":0,"Launch Time":1427397533188,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1418,"Index":416,"Attempt":0,"Launch Time":1427397533169,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533189,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":330227,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1428,"Index":426,"Attempt":0,"Launch Time":1427397533195,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1420,"Index":418,"Attempt":0,"Launch Time":1427397533178,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533195,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":627265,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1429,"Index":427,"Attempt":0,"Launch Time":1427397533196,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1424,"Index":422,"Attempt":0,"Launch Time":1427397533186,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533196,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":390670,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1430,"Index":428,"Attempt":0,"Launch Time":1427397533197,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1426,"Index":424,"Attempt":0,"Launch Time":1427397533188,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533197,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":437632,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1431,"Index":429,"Attempt":0,"Launch Time":1427397533199,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1427,"Index":425,"Attempt":0,"Launch Time":1427397533188,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533199,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":349871,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1432,"Index":430,"Attempt":0,"Launch Time":1427397533205,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1433,"Index":431,"Attempt":0,"Launch Time":1427397533205,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1429,"Index":427,"Attempt":0,"Launch Time":1427397533196,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533205,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":487273,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1430,"Index":428,"Attempt":0,"Launch Time":1427397533197,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533205,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":333281,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1434,"Index":432,"Attempt":0,"Launch Time":1427397533206,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1428,"Index":426,"Attempt":0,"Launch Time":1427397533195,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533206,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":340663,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1435,"Index":433,"Attempt":0,"Launch Time":1427397533212,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1425,"Index":423,"Attempt":0,"Launch Time":1427397533188,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533212,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":21,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":350065,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1436,"Index":434,"Attempt":0,"Launch Time":1427397533215,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1437,"Index":435,"Attempt":0,"Launch Time":1427397533215,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1434,"Index":432,"Attempt":0,"Launch Time":1427397533206,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533215,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":376933,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1432,"Index":430,"Attempt":0,"Launch Time":1427397533205,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533215,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":382027,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1438,"Index":436,"Attempt":0,"Launch Time":1427397533216,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1433,"Index":431,"Attempt":0,"Launch Time":1427397533205,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533216,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":363449,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1439,"Index":437,"Attempt":0,"Launch Time":1427397533216,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1417,"Index":415,"Attempt":0,"Launch Time":1427397533166,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533216,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":50,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":352461,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1440,"Index":438,"Attempt":0,"Launch Time":1427397533222,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1435,"Index":433,"Attempt":0,"Launch Time":1427397533212,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533222,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":393461,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1441,"Index":439,"Attempt":0,"Launch Time":1427397533224,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1438,"Index":436,"Attempt":0,"Launch Time":1427397533216,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533224,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":371597,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1442,"Index":440,"Attempt":0,"Launch Time":1427397533224,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1439,"Index":437,"Attempt":0,"Launch Time":1427397533216,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533224,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":392440,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1443,"Index":441,"Attempt":0,"Launch Time":1427397533227,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1431,"Index":429,"Attempt":0,"Launch Time":1427397533199,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533227,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":28,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":369102,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1444,"Index":442,"Attempt":0,"Launch Time":1427397533228,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1436,"Index":434,"Attempt":0,"Launch Time":1427397533215,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533228,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":402180,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1445,"Index":443,"Attempt":0,"Launch Time":1427397533228,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1437,"Index":435,"Attempt":0,"Launch Time":1427397533215,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533228,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":461008,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1446,"Index":444,"Attempt":0,"Launch Time":1427397533242,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1423,"Index":421,"Attempt":0,"Launch Time":1427397533184,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533242,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":54,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":364333,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1447,"Index":445,"Attempt":0,"Launch Time":1427397533242,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1442,"Index":440,"Attempt":0,"Launch Time":1427397533224,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533242,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":6,"Executor Run Time":12,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":329833,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1448,"Index":446,"Attempt":0,"Launch Time":1427397533247,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1443,"Index":441,"Attempt":0,"Launch Time":1427397533227,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533248,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":20,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":292242,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1449,"Index":447,"Attempt":0,"Launch Time":1427397533248,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1444,"Index":442,"Attempt":0,"Launch Time":1427397533228,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533248,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":20,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":327816,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1450,"Index":448,"Attempt":0,"Launch Time":1427397533249,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1447,"Index":445,"Attempt":0,"Launch Time":1427397533242,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533250,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9605,"Shuffle Write Time":334842,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1451,"Index":449,"Attempt":0,"Launch Time":1427397533250,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1445,"Index":443,"Attempt":0,"Launch Time":1427397533228,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533250,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":20,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":582785,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1452,"Index":450,"Attempt":0,"Launch Time":1427397533256,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1448,"Index":446,"Attempt":0,"Launch Time":1427397533247,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533256,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":355113,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1453,"Index":451,"Attempt":0,"Launch Time":1427397533258,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1449,"Index":447,"Attempt":0,"Launch Time":1427397533248,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533258,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":328457,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1454,"Index":452,"Attempt":0,"Launch Time":1427397533259,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1455,"Index":453,"Attempt":0,"Launch Time":1427397533260,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1451,"Index":449,"Attempt":0,"Launch Time":1427397533250,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533260,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":535263,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1441,"Index":439,"Attempt":0,"Launch Time":1427397533224,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533260,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":35,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":597186,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1456,"Index":454,"Attempt":0,"Launch Time":1427397533264,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1452,"Index":450,"Attempt":0,"Launch Time":1427397533256,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533264,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":352354,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1457,"Index":455,"Attempt":0,"Launch Time":1427397533265,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1446,"Index":444,"Attempt":0,"Launch Time":1427397533242,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533265,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":654543,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1458,"Index":456,"Attempt":0,"Launch Time":1427397533267,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1454,"Index":452,"Attempt":0,"Launch Time":1427397533259,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533267,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":331810,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1459,"Index":457,"Attempt":0,"Launch Time":1427397533268,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1453,"Index":451,"Attempt":0,"Launch Time":1427397533258,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533268,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":335060,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1460,"Index":458,"Attempt":0,"Launch Time":1427397533269,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1455,"Index":453,"Attempt":0,"Launch Time":1427397533260,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533270,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":366404,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1461,"Index":459,"Attempt":0,"Launch Time":1427397533273,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1456,"Index":454,"Attempt":0,"Launch Time":1427397533264,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533273,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":358778,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1462,"Index":460,"Attempt":0,"Launch Time":1427397533277,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1459,"Index":457,"Attempt":0,"Launch Time":1427397533268,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533277,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":505518,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1463,"Index":461,"Attempt":0,"Launch Time":1427397533279,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1457,"Index":455,"Attempt":0,"Launch Time":1427397533265,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533280,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1275280,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1464,"Index":462,"Attempt":0,"Launch Time":1427397533282,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1465,"Index":463,"Attempt":0,"Launch Time":1427397533282,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1460,"Index":458,"Attempt":0,"Launch Time":1427397533269,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533282,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":525255,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1440,"Index":438,"Attempt":0,"Launch Time":1427397533222,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533282,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":7,"Executor Run Time":42,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1840075,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1466,"Index":464,"Attempt":0,"Launch Time":1427397533282,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1458,"Index":456,"Attempt":0,"Launch Time":1427397533267,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533282,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":983522,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1467,"Index":465,"Attempt":0,"Launch Time":1427397533283,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1461,"Index":459,"Attempt":0,"Launch Time":1427397533273,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533283,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":414864,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1468,"Index":466,"Attempt":0,"Launch Time":1427397533285,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1450,"Index":448,"Attempt":0,"Launch Time":1427397533249,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533286,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":36,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":694355,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1462,"Index":460,"Attempt":0,"Launch Time":1427397533277,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533286,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":377719,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1469,"Index":467,"Attempt":0,"Launch Time":1427397533286,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1470,"Index":468,"Attempt":0,"Launch Time":1427397533289,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1463,"Index":461,"Attempt":0,"Launch Time":1427397533279,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533289,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":365229,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1471,"Index":469,"Attempt":0,"Launch Time":1427397533290,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1466,"Index":464,"Attempt":0,"Launch Time":1427397533282,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533290,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":309474,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1472,"Index":470,"Attempt":0,"Launch Time":1427397533292,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1467,"Index":465,"Attempt":0,"Launch Time":1427397533283,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533292,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":312866,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1473,"Index":471,"Attempt":0,"Launch Time":1427397533293,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1468,"Index":466,"Attempt":0,"Launch Time":1427397533285,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533293,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":346312,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1474,"Index":472,"Attempt":0,"Launch Time":1427397533297,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1470,"Index":468,"Attempt":0,"Launch Time":1427397533289,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533297,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":339872,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1475,"Index":473,"Attempt":0,"Launch Time":1427397533299,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1464,"Index":462,"Attempt":0,"Launch Time":1427397533282,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533299,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":328539,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1476,"Index":474,"Attempt":0,"Launch Time":1427397533299,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1477,"Index":475,"Attempt":0,"Launch Time":1427397533300,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1471,"Index":469,"Attempt":0,"Launch Time":1427397533290,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533300,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":317562,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1472,"Index":470,"Attempt":0,"Launch Time":1427397533292,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533300,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":325825,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1478,"Index":476,"Attempt":0,"Launch Time":1427397533301,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1473,"Index":471,"Attempt":0,"Launch Time":1427397533293,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533301,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9613,"Shuffle Write Time":318456,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1479,"Index":477,"Attempt":0,"Launch Time":1427397533304,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1474,"Index":472,"Attempt":0,"Launch Time":1427397533297,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533304,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":280242,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1480,"Index":478,"Attempt":0,"Launch Time":1427397533307,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1481,"Index":479,"Attempt":0,"Launch Time":1427397533307,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1476,"Index":474,"Attempt":0,"Launch Time":1427397533299,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533307,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":330939,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1477,"Index":475,"Attempt":0,"Launch Time":1427397533300,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533307,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":307210,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1482,"Index":480,"Attempt":0,"Launch Time":1427397533310,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1465,"Index":463,"Attempt":0,"Launch Time":1427397533282,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533310,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":26,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9607,"Shuffle Write Time":4263497,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1483,"Index":481,"Attempt":0,"Launch Time":1427397533310,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533323,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1478,"Index":476,"Attempt":0,"Launch Time":1427397533301,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533310,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":328874,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1484,"Index":482,"Attempt":0,"Launch Time":1427397533312,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1479,"Index":477,"Attempt":0,"Launch Time":1427397533304,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533312,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":352310,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1485,"Index":483,"Attempt":0,"Launch Time":1427397533313,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533321,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1419,"Index":417,"Attempt":0,"Launch Time":1427397533173,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533313,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":140,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":374554,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1486,"Index":484,"Attempt":0,"Launch Time":1427397533321,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1485,"Index":483,"Attempt":0,"Launch Time":1427397533313,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533321,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":373196,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1487,"Index":485,"Attempt":0,"Launch Time":1427397533323,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533352,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1483,"Index":481,"Attempt":0,"Launch Time":1427397533310,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533323,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":356131,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1488,"Index":486,"Attempt":0,"Launch Time":1427397533326,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1475,"Index":473,"Attempt":0,"Launch Time":1427397533299,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533326,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":369607,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1489,"Index":487,"Attempt":0,"Launch Time":1427397533352,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1487,"Index":485,"Attempt":0,"Launch Time":1427397533323,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533352,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":27,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":2121695,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1490,"Index":488,"Attempt":0,"Launch Time":1427397533352,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1491,"Index":489,"Attempt":0,"Launch Time":1427397533352,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1492,"Index":490,"Attempt":0,"Launch Time":1427397533353,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1493,"Index":491,"Attempt":0,"Launch Time":1427397533353,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1481,"Index":479,"Attempt":0,"Launch Time":1427397533307,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533353,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":45,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":348056,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1469,"Index":467,"Attempt":0,"Launch Time":1427397533286,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533353,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":60,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":328631,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1480,"Index":478,"Attempt":0,"Launch Time":1427397533307,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533354,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":45,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":359118,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1486,"Index":484,"Attempt":0,"Launch Time":1427397533321,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533354,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":24,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":520761,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1494,"Index":492,"Attempt":0,"Launch Time":1427397533355,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1484,"Index":482,"Attempt":0,"Launch Time":1427397533312,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533355,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":7,"Executor Run Time":33,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":368866,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1495,"Index":493,"Attempt":0,"Launch Time":1427397533360,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1489,"Index":487,"Attempt":0,"Launch Time":1427397533352,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533360,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":321373,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1496,"Index":494,"Attempt":0,"Launch Time":1427397533361,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1493,"Index":491,"Attempt":0,"Launch Time":1427397533353,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533361,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":588233,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1497,"Index":495,"Attempt":0,"Launch Time":1427397533362,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1492,"Index":490,"Attempt":0,"Launch Time":1427397533353,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533362,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":383100,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1498,"Index":496,"Attempt":0,"Launch Time":1427397533364,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1494,"Index":492,"Attempt":0,"Launch Time":1427397533355,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533364,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":347783,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1499,"Index":497,"Attempt":0,"Launch Time":1427397533364,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1488,"Index":486,"Attempt":0,"Launch Time":1427397533326,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533364,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":30,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":364094,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1500,"Index":498,"Attempt":0,"Launch Time":1427397533369,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1490,"Index":488,"Attempt":0,"Launch Time":1427397533352,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533369,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":491123,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1495,"Index":493,"Attempt":0,"Launch Time":1427397533360,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533369,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":355489,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1501,"Index":499,"Attempt":0,"Launch Time":1427397533369,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1502,"Index":500,"Attempt":0,"Launch Time":1427397533370,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1496,"Index":494,"Attempt":0,"Launch Time":1427397533361,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533370,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":363718,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1503,"Index":501,"Attempt":0,"Launch Time":1427397533370,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1497,"Index":495,"Attempt":0,"Launch Time":1427397533362,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533370,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":355447,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1504,"Index":502,"Attempt":0,"Launch Time":1427397533374,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1498,"Index":496,"Attempt":0,"Launch Time":1427397533364,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533374,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1540396,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1505,"Index":503,"Attempt":0,"Launch Time":1427397533378,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1500,"Index":498,"Attempt":0,"Launch Time":1427397533369,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533378,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":376360,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1502,"Index":500,"Attempt":0,"Launch Time":1427397533370,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533378,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":495888,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1506,"Index":504,"Attempt":0,"Launch Time":1427397533378,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1507,"Index":505,"Attempt":0,"Launch Time":1427397533380,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1503,"Index":501,"Attempt":0,"Launch Time":1427397533370,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533380,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":414878,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1508,"Index":506,"Attempt":0,"Launch Time":1427397533381,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1501,"Index":499,"Attempt":0,"Launch Time":1427397533369,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533381,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":357547,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1509,"Index":507,"Attempt":0,"Launch Time":1427397533383,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1504,"Index":502,"Attempt":0,"Launch Time":1427397533374,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533383,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":334929,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1510,"Index":508,"Attempt":0,"Launch Time":1427397533386,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1506,"Index":504,"Attempt":0,"Launch Time":1427397533378,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533386,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":327462,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1511,"Index":509,"Attempt":0,"Launch Time":1427397533387,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1505,"Index":503,"Attempt":0,"Launch Time":1427397533378,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533387,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":328936,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1512,"Index":510,"Attempt":0,"Launch Time":1427397533388,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1499,"Index":497,"Attempt":0,"Launch Time":1427397533364,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533388,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":331970,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1513,"Index":511,"Attempt":0,"Launch Time":1427397533390,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1507,"Index":505,"Attempt":0,"Launch Time":1427397533380,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533390,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":356966,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1514,"Index":512,"Attempt":0,"Launch Time":1427397533393,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1509,"Index":507,"Attempt":0,"Launch Time":1427397533383,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533393,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":493261,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1515,"Index":513,"Attempt":0,"Launch Time":1427397533394,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1510,"Index":508,"Attempt":0,"Launch Time":1427397533386,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533394,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":364770,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1516,"Index":514,"Attempt":0,"Launch Time":1427397533395,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1508,"Index":506,"Attempt":0,"Launch Time":1427397533381,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533395,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":362448,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1517,"Index":515,"Attempt":0,"Launch Time":1427397533396,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1511,"Index":509,"Attempt":0,"Launch Time":1427397533387,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533396,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":337496,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1518,"Index":516,"Attempt":0,"Launch Time":1427397533401,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1515,"Index":513,"Attempt":0,"Launch Time":1427397533394,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533401,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":297753,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1519,"Index":517,"Attempt":0,"Launch Time":1427397533402,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1513,"Index":511,"Attempt":0,"Launch Time":1427397533390,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533402,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":627412,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1520,"Index":518,"Attempt":0,"Launch Time":1427397533404,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1517,"Index":515,"Attempt":0,"Launch Time":1427397533396,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533404,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":341201,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1521,"Index":519,"Attempt":0,"Launch Time":1427397533406,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1491,"Index":489,"Attempt":0,"Launch Time":1427397533352,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533406,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":48,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1455317,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1522,"Index":520,"Attempt":0,"Launch Time":1427397533408,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1518,"Index":516,"Attempt":0,"Launch Time":1427397533401,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533409,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":351365,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1523,"Index":521,"Attempt":0,"Launch Time":1427397533409,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1516,"Index":514,"Attempt":0,"Launch Time":1427397533395,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533409,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1003257,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1524,"Index":522,"Attempt":0,"Launch Time":1427397533409,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1514,"Index":512,"Attempt":0,"Launch Time":1427397533393,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533410,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":341031,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1525,"Index":523,"Attempt":0,"Launch Time":1427397533411,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1512,"Index":510,"Attempt":0,"Launch Time":1427397533388,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533411,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":22,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":367585,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1526,"Index":524,"Attempt":0,"Launch Time":1427397533412,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1520,"Index":518,"Attempt":0,"Launch Time":1427397533404,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533412,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":340620,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1527,"Index":525,"Attempt":0,"Launch Time":1427397533415,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1519,"Index":517,"Attempt":0,"Launch Time":1427397533402,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533415,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":415972,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1528,"Index":526,"Attempt":0,"Launch Time":1427397533417,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1522,"Index":520,"Attempt":0,"Launch Time":1427397533408,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533418,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":361274,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1529,"Index":527,"Attempt":0,"Launch Time":1427397533418,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1530,"Index":528,"Attempt":0,"Launch Time":1427397533418,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1524,"Index":522,"Attempt":0,"Launch Time":1427397533409,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533418,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":826079,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1523,"Index":521,"Attempt":0,"Launch Time":1427397533409,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533418,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":382574,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1531,"Index":529,"Attempt":0,"Launch Time":1427397533421,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1532,"Index":530,"Attempt":0,"Launch Time":1427397533423,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533457,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1533,"Index":531,"Attempt":0,"Launch Time":1427397533430,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533479,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1525,"Index":523,"Attempt":0,"Launch Time":1427397533411,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533430,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":352285,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1526,"Index":524,"Attempt":0,"Launch Time":1427397533412,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533432,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":368361,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1534,"Index":532,"Attempt":0,"Launch Time":1427397533434,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533477,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1529,"Index":527,"Attempt":0,"Launch Time":1427397533418,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533435,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":369849,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1535,"Index":533,"Attempt":0,"Launch Time":1427397533437,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1530,"Index":528,"Attempt":0,"Launch Time":1427397533418,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533437,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":373999,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1536,"Index":534,"Attempt":0,"Launch Time":1427397533443,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533453,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1521,"Index":519,"Attempt":0,"Launch Time":1427397533406,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533443,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":9,"Executor Run Time":26,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":348897,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1537,"Index":535,"Attempt":0,"Launch Time":1427397533443,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533451,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1482,"Index":480,"Attempt":0,"Launch Time":1427397533310,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533443,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":110,"Executor Run Time":7,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":352693,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1538,"Index":536,"Attempt":0,"Launch Time":1427397533451,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533459,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1537,"Index":535,"Attempt":0,"Launch Time":1427397533443,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533451,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":360173,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1539,"Index":537,"Attempt":0,"Launch Time":1427397533453,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533460,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1536,"Index":534,"Attempt":0,"Launch Time":1427397533443,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533453,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":392809,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1540,"Index":538,"Attempt":0,"Launch Time":1427397533457,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533467,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1532,"Index":530,"Attempt":0,"Launch Time":1427397533423,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533457,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":27,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":352641,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1541,"Index":539,"Attempt":0,"Launch Time":1427397533459,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533467,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1538,"Index":536,"Attempt":0,"Launch Time":1427397533451,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533459,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":357736,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1542,"Index":540,"Attempt":0,"Launch Time":1427397533460,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533473,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1539,"Index":537,"Attempt":0,"Launch Time":1427397533453,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533460,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":362853,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1543,"Index":541,"Attempt":0,"Launch Time":1427397533466,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533474,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1544,"Index":542,"Attempt":0,"Launch Time":1427397533467,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533478,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1540,"Index":538,"Attempt":0,"Launch Time":1427397533457,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533467,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":344663,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1541,"Index":539,"Attempt":0,"Launch Time":1427397533459,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533467,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":339310,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1545,"Index":543,"Attempt":0,"Launch Time":1427397533473,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533482,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1542,"Index":540,"Attempt":0,"Launch Time":1427397533460,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533473,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":351568,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1546,"Index":544,"Attempt":0,"Launch Time":1427397533474,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533483,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1543,"Index":541,"Attempt":0,"Launch Time":1427397533466,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533474,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":359733,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1547,"Index":545,"Attempt":0,"Launch Time":1427397533476,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533493,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1534,"Index":532,"Attempt":0,"Launch Time":1427397533434,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533477,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":40,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":472626,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1548,"Index":546,"Attempt":0,"Launch Time":1427397533478,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533486,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1544,"Index":542,"Attempt":0,"Launch Time":1427397533467,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533478,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":800000,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1549,"Index":547,"Attempt":0,"Launch Time":1427397533479,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533489,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1533,"Index":531,"Attempt":0,"Launch Time":1427397533430,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533479,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":48,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":386153,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1550,"Index":548,"Attempt":0,"Launch Time":1427397533482,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533491,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1545,"Index":543,"Attempt":0,"Launch Time":1427397533473,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533482,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":426912,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1551,"Index":549,"Attempt":0,"Launch Time":1427397533483,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533492,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1546,"Index":544,"Attempt":0,"Launch Time":1427397533474,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533483,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":390265,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1552,"Index":550,"Attempt":0,"Launch Time":1427397533486,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533495,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1548,"Index":546,"Attempt":0,"Launch Time":1427397533478,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533486,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":318421,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1553,"Index":551,"Attempt":0,"Launch Time":1427397533489,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533507,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1549,"Index":547,"Attempt":0,"Launch Time":1427397533479,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533489,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":352960,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1554,"Index":552,"Attempt":0,"Launch Time":1427397533491,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533499,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1550,"Index":548,"Attempt":0,"Launch Time":1427397533482,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533491,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":377898,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1555,"Index":553,"Attempt":0,"Launch Time":1427397533491,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533499,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1551,"Index":549,"Attempt":0,"Launch Time":1427397533483,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533492,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":415748,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1556,"Index":554,"Attempt":0,"Launch Time":1427397533493,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533500,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1547,"Index":545,"Attempt":0,"Launch Time":1427397533476,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533493,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":837592,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1557,"Index":555,"Attempt":0,"Launch Time":1427397533494,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533502,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1552,"Index":550,"Attempt":0,"Launch Time":1427397533486,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533495,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":334254,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1558,"Index":556,"Attempt":0,"Launch Time":1427397533499,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533508,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1555,"Index":553,"Attempt":0,"Launch Time":1427397533491,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533499,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":341120,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1559,"Index":557,"Attempt":0,"Launch Time":1427397533499,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533508,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1554,"Index":552,"Attempt":0,"Launch Time":1427397533491,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533499,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":323104,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1560,"Index":558,"Attempt":0,"Launch Time":1427397533500,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533508,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1556,"Index":554,"Attempt":0,"Launch Time":1427397533493,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533500,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":328716,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1561,"Index":559,"Attempt":0,"Launch Time":1427397533502,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533510,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1557,"Index":555,"Attempt":0,"Launch Time":1427397533494,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533502,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":311725,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1562,"Index":560,"Attempt":0,"Launch Time":1427397533506,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533519,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1553,"Index":551,"Attempt":0,"Launch Time":1427397533489,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533507,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":394559,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1563,"Index":561,"Attempt":0,"Launch Time":1427397533507,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533516,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1559,"Index":557,"Attempt":0,"Launch Time":1427397533499,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533508,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":381944,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1564,"Index":562,"Attempt":0,"Launch Time":1427397533508,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533516,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1560,"Index":558,"Attempt":0,"Launch Time":1427397533500,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533508,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":389315,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1565,"Index":563,"Attempt":0,"Launch Time":1427397533508,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533516,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1558,"Index":556,"Attempt":0,"Launch Time":1427397533499,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533508,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":508780,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1566,"Index":564,"Attempt":0,"Launch Time":1427397533510,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533518,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1561,"Index":559,"Attempt":0,"Launch Time":1427397533502,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533510,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":413015,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1567,"Index":565,"Attempt":0,"Launch Time":1427397533514,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1528,"Index":526,"Attempt":0,"Launch Time":1427397533417,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533514,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":95,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1070386,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1568,"Index":566,"Attempt":0,"Launch Time":1427397533516,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533531,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1569,"Index":567,"Attempt":0,"Launch Time":1427397533516,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533530,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1563,"Index":561,"Attempt":0,"Launch Time":1427397533507,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533516,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":443773,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1564,"Index":562,"Attempt":0,"Launch Time":1427397533508,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533516,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":383173,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1570,"Index":568,"Attempt":0,"Launch Time":1427397533516,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533530,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1565,"Index":563,"Attempt":0,"Launch Time":1427397533508,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533516,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":347911,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1571,"Index":569,"Attempt":0,"Launch Time":1427397533518,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533530,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1566,"Index":564,"Attempt":0,"Launch Time":1427397533510,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533518,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":335235,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1572,"Index":570,"Attempt":0,"Launch Time":1427397533519,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533531,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1562,"Index":560,"Attempt":0,"Launch Time":1427397533506,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533519,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":448241,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1573,"Index":571,"Attempt":0,"Launch Time":1427397533523,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1574,"Index":572,"Attempt":0,"Launch Time":1427397533523,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1575,"Index":573,"Attempt":0,"Launch Time":1427397533523,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1576,"Index":574,"Attempt":0,"Launch Time":1427397533530,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1577,"Index":575,"Attempt":0,"Launch Time":1427397533530,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1570,"Index":568,"Attempt":0,"Launch Time":1427397533516,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533530,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":301216,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1569,"Index":567,"Attempt":0,"Launch Time":1427397533516,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533530,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":336027,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1571,"Index":569,"Attempt":0,"Launch Time":1427397533518,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533530,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":333192,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1568,"Index":566,"Attempt":0,"Launch Time":1427397533516,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533531,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":321353,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1572,"Index":570,"Attempt":0,"Launch Time":1427397533519,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533531,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":361171,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1527,"Index":525,"Attempt":0,"Launch Time":1427397533415,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533545,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":355515,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1578,"Index":576,"Attempt":0,"Launch Time":1427397533550,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1574,"Index":572,"Attempt":0,"Launch Time":1427397533523,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533550,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":21,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9614,"Shuffle Write Time":2362919,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1579,"Index":577,"Attempt":0,"Launch Time":1427397533561,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1578,"Index":576,"Attempt":0,"Launch Time":1427397533550,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533561,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":923128,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1580,"Index":578,"Attempt":0,"Launch Time":1427397533562,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1576,"Index":574,"Attempt":0,"Launch Time":1427397533530,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533562,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":30,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":432327,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1581,"Index":579,"Attempt":0,"Launch Time":1427397533568,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533576,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1531,"Index":529,"Attempt":0,"Launch Time":1427397533421,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533568,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":7,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":357152,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1582,"Index":580,"Attempt":0,"Launch Time":1427397533569,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533594,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1535,"Index":533,"Attempt":0,"Launch Time":1427397533437,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533569,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":345776,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1583,"Index":581,"Attempt":0,"Launch Time":1427397533572,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1567,"Index":565,"Attempt":0,"Launch Time":1427397533514,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533573,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":14,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":340877,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1584,"Index":582,"Attempt":0,"Launch Time":1427397533576,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1581,"Index":579,"Attempt":0,"Launch Time":1427397533568,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533576,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":338333,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1585,"Index":583,"Attempt":0,"Launch Time":1427397533593,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1582,"Index":580,"Attempt":0,"Launch Time":1427397533569,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533594,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9611,"Shuffle Write Time":337520,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1586,"Index":584,"Attempt":0,"Launch Time":1427397533594,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1583,"Index":581,"Attempt":0,"Launch Time":1427397533572,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533594,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":316910,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1587,"Index":585,"Attempt":0,"Launch Time":1427397533594,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1588,"Index":586,"Attempt":0,"Launch Time":1427397533595,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1575,"Index":573,"Attempt":0,"Launch Time":1427397533523,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533595,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":27,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":706428,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1580,"Index":578,"Attempt":0,"Launch Time":1427397533562,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533595,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":346970,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1584,"Index":582,"Attempt":0,"Launch Time":1427397533576,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533595,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":370176,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1589,"Index":587,"Attempt":0,"Launch Time":1427397533595,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1590,"Index":588,"Attempt":0,"Launch Time":1427397533595,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1591,"Index":589,"Attempt":0,"Launch Time":1427397533595,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1573,"Index":571,"Attempt":0,"Launch Time":1427397533523,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533595,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":27,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":360554,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1577,"Index":575,"Attempt":0,"Launch Time":1427397533530,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533596,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":25,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":367119,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1592,"Index":590,"Attempt":0,"Launch Time":1427397533602,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1585,"Index":583,"Attempt":0,"Launch Time":1427397533593,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533602,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":340352,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1593,"Index":591,"Attempt":0,"Launch Time":1427397533604,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1589,"Index":587,"Attempt":0,"Launch Time":1427397533595,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533604,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":373016,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1594,"Index":592,"Attempt":0,"Launch Time":1427397533604,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1588,"Index":586,"Attempt":0,"Launch Time":1427397533595,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533604,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":986589,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1595,"Index":593,"Attempt":0,"Launch Time":1427397533605,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1590,"Index":588,"Attempt":0,"Launch Time":1427397533595,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533605,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9608,"Shuffle Write Time":331220,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1596,"Index":594,"Attempt":0,"Launch Time":1427397533605,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1591,"Index":589,"Attempt":0,"Launch Time":1427397533595,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533606,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":514371,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1597,"Index":595,"Attempt":0,"Launch Time":1427397533609,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1592,"Index":590,"Attempt":0,"Launch Time":1427397533602,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533610,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":385523,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1598,"Index":596,"Attempt":0,"Launch Time":1427397533614,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1594,"Index":592,"Attempt":0,"Launch Time":1427397533604,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533614,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":398693,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1599,"Index":597,"Attempt":0,"Launch Time":1427397533615,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1593,"Index":591,"Attempt":0,"Launch Time":1427397533604,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533615,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":447900,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1600,"Index":598,"Attempt":0,"Launch Time":1427397533616,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1595,"Index":593,"Attempt":0,"Launch Time":1427397533605,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533616,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":408202,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1601,"Index":599,"Attempt":0,"Launch Time":1427397533616,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1596,"Index":594,"Attempt":0,"Launch Time":1427397533605,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533616,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":424230,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1602,"Index":600,"Attempt":0,"Launch Time":1427397533617,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1597,"Index":595,"Attempt":0,"Launch Time":1427397533609,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533618,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":368765,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1603,"Index":601,"Attempt":0,"Launch Time":1427397533623,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1598,"Index":596,"Attempt":0,"Launch Time":1427397533614,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533623,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":339519,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1604,"Index":602,"Attempt":0,"Launch Time":1427397533624,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1600,"Index":598,"Attempt":0,"Launch Time":1427397533616,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533624,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":351172,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1605,"Index":603,"Attempt":0,"Launch Time":1427397533627,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1602,"Index":600,"Attempt":0,"Launch Time":1427397533617,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533627,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":370658,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1606,"Index":604,"Attempt":0,"Launch Time":1427397533627,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1599,"Index":597,"Attempt":0,"Launch Time":1427397533615,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533627,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":362640,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1607,"Index":605,"Attempt":0,"Launch Time":1427397533628,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1601,"Index":599,"Attempt":0,"Launch Time":1427397533616,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533629,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":369260,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1608,"Index":606,"Attempt":0,"Launch Time":1427397533632,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1603,"Index":601,"Attempt":0,"Launch Time":1427397533623,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533632,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":394681,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1609,"Index":607,"Attempt":0,"Launch Time":1427397533632,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1604,"Index":602,"Attempt":0,"Launch Time":1427397533624,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533632,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":336303,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1610,"Index":608,"Attempt":0,"Launch Time":1427397533634,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1606,"Index":604,"Attempt":0,"Launch Time":1427397533627,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533634,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":331170,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1611,"Index":609,"Attempt":0,"Launch Time":1427397533636,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1607,"Index":605,"Attempt":0,"Launch Time":1427397533628,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533637,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":334744,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1612,"Index":610,"Attempt":0,"Launch Time":1427397533640,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1609,"Index":607,"Attempt":0,"Launch Time":1427397533632,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533640,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":351204,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1613,"Index":611,"Attempt":0,"Launch Time":1427397533642,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1610,"Index":608,"Attempt":0,"Launch Time":1427397533634,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533642,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":297136,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1614,"Index":612,"Attempt":0,"Launch Time":1427397533642,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1608,"Index":606,"Attempt":0,"Launch Time":1427397533632,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533642,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":328745,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1615,"Index":613,"Attempt":0,"Launch Time":1427397533646,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1611,"Index":609,"Attempt":0,"Launch Time":1427397533636,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533647,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":343063,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1616,"Index":614,"Attempt":0,"Launch Time":1427397533648,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1612,"Index":610,"Attempt":0,"Launch Time":1427397533640,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533648,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":335183,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1617,"Index":615,"Attempt":0,"Launch Time":1427397533649,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1587,"Index":585,"Attempt":0,"Launch Time":1427397533594,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533649,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":31,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1359105,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1618,"Index":616,"Attempt":0,"Launch Time":1427397533650,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1613,"Index":611,"Attempt":0,"Launch Time":1427397533642,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533650,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":361411,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1619,"Index":617,"Attempt":0,"Launch Time":1427397533651,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1614,"Index":612,"Attempt":0,"Launch Time":1427397533642,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533651,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":402396,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1620,"Index":618,"Attempt":0,"Launch Time":1427397533661,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1616,"Index":614,"Attempt":0,"Launch Time":1427397533648,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533661,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":370169,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1621,"Index":619,"Attempt":0,"Launch Time":1427397533664,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1619,"Index":617,"Attempt":0,"Launch Time":1427397533651,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533665,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":372583,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1622,"Index":620,"Attempt":0,"Launch Time":1427397533666,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1615,"Index":613,"Attempt":0,"Launch Time":1427397533646,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533666,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":14,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":380709,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1623,"Index":621,"Attempt":0,"Launch Time":1427397533667,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1617,"Index":615,"Attempt":0,"Launch Time":1427397533649,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533667,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":339725,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1624,"Index":622,"Attempt":0,"Launch Time":1427397533669,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1618,"Index":616,"Attempt":0,"Launch Time":1427397533650,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533669,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":19,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":349545,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1625,"Index":623,"Attempt":0,"Launch Time":1427397533670,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1620,"Index":618,"Attempt":0,"Launch Time":1427397533661,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533670,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":375652,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1626,"Index":624,"Attempt":0,"Launch Time":1427397533674,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1622,"Index":620,"Attempt":0,"Launch Time":1427397533666,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533674,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":374657,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1627,"Index":625,"Attempt":0,"Launch Time":1427397533674,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1605,"Index":603,"Attempt":0,"Launch Time":1427397533627,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533674,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":44,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":329375,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1628,"Index":626,"Attempt":0,"Launch Time":1427397533676,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1621,"Index":619,"Attempt":0,"Launch Time":1427397533664,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533676,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":334209,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1629,"Index":627,"Attempt":0,"Launch Time":1427397533677,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1624,"Index":622,"Attempt":0,"Launch Time":1427397533669,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533677,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":361478,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1630,"Index":628,"Attempt":0,"Launch Time":1427397533679,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1625,"Index":623,"Attempt":0,"Launch Time":1427397533670,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533679,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":433068,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1631,"Index":629,"Attempt":0,"Launch Time":1427397533684,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1632,"Index":630,"Attempt":0,"Launch Time":1427397533684,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1627,"Index":625,"Attempt":0,"Launch Time":1427397533674,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533684,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":422579,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1628,"Index":626,"Attempt":0,"Launch Time":1427397533676,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533684,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":362948,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1633,"Index":631,"Attempt":0,"Launch Time":1427397533685,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1629,"Index":627,"Attempt":0,"Launch Time":1427397533677,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533685,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":350618,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1634,"Index":632,"Attempt":0,"Launch Time":1427397533691,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1635,"Index":633,"Attempt":0,"Launch Time":1427397533691,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1586,"Index":584,"Attempt":0,"Launch Time":1427397533594,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533691,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":22,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":338560,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1632,"Index":630,"Attempt":0,"Launch Time":1427397533684,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533691,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":347538,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1636,"Index":634,"Attempt":0,"Launch Time":1427397533699,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1634,"Index":632,"Attempt":0,"Launch Time":1427397533691,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533699,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":364520,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1637,"Index":635,"Attempt":0,"Launch Time":1427397533700,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1579,"Index":577,"Attempt":0,"Launch Time":1427397533561,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533700,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":327993,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1638,"Index":636,"Attempt":0,"Launch Time":1427397533702,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1626,"Index":624,"Attempt":0,"Launch Time":1427397533674,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533702,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":27,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":882527,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1639,"Index":637,"Attempt":0,"Launch Time":1427397533709,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1636,"Index":634,"Attempt":0,"Launch Time":1427397533699,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533709,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":369312,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1640,"Index":638,"Attempt":0,"Launch Time":1427397533711,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1641,"Index":639,"Attempt":0,"Launch Time":1427397533711,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1637,"Index":635,"Attempt":0,"Launch Time":1427397533700,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533711,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":376929,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1638,"Index":636,"Attempt":0,"Launch Time":1427397533702,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533711,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":429006,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1642,"Index":640,"Attempt":0,"Launch Time":1427397533715,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1635,"Index":633,"Attempt":0,"Launch Time":1427397533691,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533715,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":380051,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1643,"Index":641,"Attempt":0,"Launch Time":1427397533718,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1639,"Index":637,"Attempt":0,"Launch Time":1427397533709,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533718,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":342021,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1644,"Index":642,"Attempt":0,"Launch Time":1427397533719,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1641,"Index":639,"Attempt":0,"Launch Time":1427397533711,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533719,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":337845,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1645,"Index":643,"Attempt":0,"Launch Time":1427397533720,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1623,"Index":621,"Attempt":0,"Launch Time":1427397533667,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533720,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":52,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":18708393,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1646,"Index":644,"Attempt":0,"Launch Time":1427397533723,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1642,"Index":640,"Attempt":0,"Launch Time":1427397533715,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533723,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":403161,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1647,"Index":645,"Attempt":0,"Launch Time":1427397533724,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1640,"Index":638,"Attempt":0,"Launch Time":1427397533711,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533724,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":358134,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1648,"Index":646,"Attempt":0,"Launch Time":1427397533725,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1643,"Index":641,"Attempt":0,"Launch Time":1427397533718,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533725,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":337604,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1649,"Index":647,"Attempt":0,"Launch Time":1427397533726,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1644,"Index":642,"Attempt":0,"Launch Time":1427397533719,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533726,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":342548,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1650,"Index":648,"Attempt":0,"Launch Time":1427397533730,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1646,"Index":644,"Attempt":0,"Launch Time":1427397533723,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533730,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":355324,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1651,"Index":649,"Attempt":0,"Launch Time":1427397533731,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1645,"Index":643,"Attempt":0,"Launch Time":1427397533720,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533732,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":352344,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1652,"Index":650,"Attempt":0,"Launch Time":1427397533732,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1648,"Index":646,"Attempt":0,"Launch Time":1427397533725,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533732,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":321327,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1653,"Index":651,"Attempt":0,"Launch Time":1427397533733,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1649,"Index":647,"Attempt":0,"Launch Time":1427397533726,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533733,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":343768,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1654,"Index":652,"Attempt":0,"Launch Time":1427397533736,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1650,"Index":648,"Attempt":0,"Launch Time":1427397533730,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533736,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":275317,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1655,"Index":653,"Attempt":0,"Launch Time":1427397533739,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1652,"Index":650,"Attempt":0,"Launch Time":1427397533732,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533739,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":289058,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1656,"Index":654,"Attempt":0,"Launch Time":1427397533740,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1653,"Index":651,"Attempt":0,"Launch Time":1427397533733,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533741,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":353856,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1657,"Index":655,"Attempt":0,"Launch Time":1427397533743,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1654,"Index":652,"Attempt":0,"Launch Time":1427397533736,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533743,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":299655,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1658,"Index":656,"Attempt":0,"Launch Time":1427397533745,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1647,"Index":645,"Attempt":0,"Launch Time":1427397533724,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533745,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":20,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":676487,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1659,"Index":657,"Attempt":0,"Launch Time":1427397533746,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1655,"Index":653,"Attempt":0,"Launch Time":1427397533739,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533746,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":344328,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1660,"Index":658,"Attempt":0,"Launch Time":1427397533748,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1656,"Index":654,"Attempt":0,"Launch Time":1427397533740,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533748,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":365413,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1661,"Index":659,"Attempt":0,"Launch Time":1427397533751,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1657,"Index":655,"Attempt":0,"Launch Time":1427397533743,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533751,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":347683,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1662,"Index":660,"Attempt":0,"Launch Time":1427397533762,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1651,"Index":649,"Attempt":0,"Launch Time":1427397533731,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533762,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":15,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":439780,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1663,"Index":661,"Attempt":0,"Launch Time":1427397533767,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1661,"Index":659,"Attempt":0,"Launch Time":1427397533751,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533767,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":446106,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1664,"Index":662,"Attempt":0,"Launch Time":1427397533768,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1659,"Index":657,"Attempt":0,"Launch Time":1427397533746,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533769,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":21,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":403687,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1665,"Index":663,"Attempt":0,"Launch Time":1427397533769,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1660,"Index":658,"Attempt":0,"Launch Time":1427397533748,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533769,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":20,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":11272360,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1666,"Index":664,"Attempt":0,"Launch Time":1427397533771,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1631,"Index":629,"Attempt":0,"Launch Time":1427397533684,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533771,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":87,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":345287,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1667,"Index":665,"Attempt":0,"Launch Time":1427397533776,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1664,"Index":662,"Attempt":0,"Launch Time":1427397533768,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533776,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":374138,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1668,"Index":666,"Attempt":0,"Launch Time":1427397533776,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1665,"Index":663,"Attempt":0,"Launch Time":1427397533769,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533776,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":351463,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1669,"Index":667,"Attempt":0,"Launch Time":1427397533778,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1662,"Index":660,"Attempt":0,"Launch Time":1427397533762,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533778,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":4437275,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1670,"Index":668,"Attempt":0,"Launch Time":1427397533778,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1666,"Index":664,"Attempt":0,"Launch Time":1427397533771,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533778,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":338145,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1671,"Index":669,"Attempt":0,"Launch Time":1427397533784,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1668,"Index":666,"Attempt":0,"Launch Time":1427397533776,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533784,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":310539,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1672,"Index":670,"Attempt":0,"Launch Time":1427397533785,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1667,"Index":665,"Attempt":0,"Launch Time":1427397533776,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533785,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":372530,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1673,"Index":671,"Attempt":0,"Launch Time":1427397533786,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1670,"Index":668,"Attempt":0,"Launch Time":1427397533778,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533786,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":373641,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1674,"Index":672,"Attempt":0,"Launch Time":1427397533794,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1675,"Index":673,"Attempt":0,"Launch Time":1427397533794,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1671,"Index":669,"Attempt":0,"Launch Time":1427397533784,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533794,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":361456,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1672,"Index":670,"Attempt":0,"Launch Time":1427397533785,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533794,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":385924,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1676,"Index":674,"Attempt":0,"Launch Time":1427397533797,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1669,"Index":667,"Attempt":0,"Launch Time":1427397533778,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533797,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":19,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":474887,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1677,"Index":675,"Attempt":0,"Launch Time":1427397533798,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1673,"Index":671,"Attempt":0,"Launch Time":1427397533786,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533798,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":404623,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1678,"Index":676,"Attempt":0,"Launch Time":1427397533802,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1674,"Index":672,"Attempt":0,"Launch Time":1427397533794,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533802,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":387847,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1679,"Index":677,"Attempt":0,"Launch Time":1427397533803,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1675,"Index":673,"Attempt":0,"Launch Time":1427397533794,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533803,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":599603,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1680,"Index":678,"Attempt":0,"Launch Time":1427397533807,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1663,"Index":661,"Attempt":0,"Launch Time":1427397533767,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533807,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":40,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":389111,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1681,"Index":679,"Attempt":0,"Launch Time":1427397533811,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1676,"Index":674,"Attempt":0,"Launch Time":1427397533797,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533811,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":447602,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1682,"Index":680,"Attempt":0,"Launch Time":1427397533813,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1677,"Index":675,"Attempt":0,"Launch Time":1427397533798,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533813,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":498335,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1683,"Index":681,"Attempt":0,"Launch Time":1427397533813,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1679,"Index":677,"Attempt":0,"Launch Time":1427397533803,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533814,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":471103,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1684,"Index":682,"Attempt":0,"Launch Time":1427397533814,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1630,"Index":628,"Attempt":0,"Launch Time":1427397533679,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533814,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":135,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":365616,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1685,"Index":683,"Attempt":0,"Launch Time":1427397533816,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1633,"Index":631,"Attempt":0,"Launch Time":1427397533685,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533816,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":131,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":587009,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1686,"Index":684,"Attempt":0,"Launch Time":1427397533816,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1658,"Index":656,"Attempt":0,"Launch Time":1427397533745,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533816,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":70,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":727149,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1687,"Index":685,"Attempt":0,"Launch Time":1427397533823,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1684,"Index":682,"Attempt":0,"Launch Time":1427397533814,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533823,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":349800,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1683,"Index":681,"Attempt":0,"Launch Time":1427397533813,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533823,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":402620,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1688,"Index":686,"Attempt":0,"Launch Time":1427397533825,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1689,"Index":687,"Attempt":0,"Launch Time":1427397533825,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1686,"Index":684,"Attempt":0,"Launch Time":1427397533816,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533825,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":362457,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1690,"Index":688,"Attempt":0,"Launch Time":1427397533830,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1685,"Index":683,"Attempt":0,"Launch Time":1427397533816,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533831,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":349501,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1691,"Index":689,"Attempt":0,"Launch Time":1427397533834,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1689,"Index":687,"Attempt":0,"Launch Time":1427397533825,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533835,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":407176,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1692,"Index":690,"Attempt":0,"Launch Time":1427397533838,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1687,"Index":685,"Attempt":0,"Launch Time":1427397533823,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533839,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":378262,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1693,"Index":691,"Attempt":0,"Launch Time":1427397533839,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1688,"Index":686,"Attempt":0,"Launch Time":1427397533825,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533839,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":431685,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1694,"Index":692,"Attempt":0,"Launch Time":1427397533842,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1691,"Index":689,"Attempt":0,"Launch Time":1427397533834,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533842,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":372085,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1695,"Index":693,"Attempt":0,"Launch Time":1427397533847,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1692,"Index":690,"Attempt":0,"Launch Time":1427397533838,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533847,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":368353,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1696,"Index":694,"Attempt":0,"Launch Time":1427397533850,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1693,"Index":691,"Attempt":0,"Launch Time":1427397533839,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533850,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":395556,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1697,"Index":695,"Attempt":0,"Launch Time":1427397533850,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1690,"Index":688,"Attempt":0,"Launch Time":1427397533830,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533851,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":381691,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1698,"Index":696,"Attempt":0,"Launch Time":1427397533851,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1694,"Index":692,"Attempt":0,"Launch Time":1427397533842,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533851,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":373137,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1699,"Index":697,"Attempt":0,"Launch Time":1427397533857,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1695,"Index":693,"Attempt":0,"Launch Time":1427397533847,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533857,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":410577,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1700,"Index":698,"Attempt":0,"Launch Time":1427397533860,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1697,"Index":695,"Attempt":0,"Launch Time":1427397533850,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533860,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":410688,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1701,"Index":699,"Attempt":0,"Launch Time":1427397533860,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1698,"Index":696,"Attempt":0,"Launch Time":1427397533851,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533860,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":385146,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1702,"Index":700,"Attempt":0,"Launch Time":1427397533864,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1696,"Index":694,"Attempt":0,"Launch Time":1427397533850,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533864,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":428999,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1703,"Index":701,"Attempt":0,"Launch Time":1427397533865,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1699,"Index":697,"Attempt":0,"Launch Time":1427397533857,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533865,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9612,"Shuffle Write Time":410550,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1704,"Index":702,"Attempt":0,"Launch Time":1427397533868,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1681,"Index":679,"Attempt":0,"Launch Time":1427397533811,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533868,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":8,"Executor Run Time":42,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":655699,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1705,"Index":703,"Attempt":0,"Launch Time":1427397533868,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1700,"Index":698,"Attempt":0,"Launch Time":1427397533860,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533869,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":375957,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1706,"Index":704,"Attempt":0,"Launch Time":1427397533870,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1701,"Index":699,"Attempt":0,"Launch Time":1427397533860,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533870,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":364799,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1707,"Index":705,"Attempt":0,"Launch Time":1427397533879,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1702,"Index":700,"Attempt":0,"Launch Time":1427397533864,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533879,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":365098,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1708,"Index":706,"Attempt":0,"Launch Time":1427397533880,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1703,"Index":701,"Attempt":0,"Launch Time":1427397533865,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533880,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":14,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":360588,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1709,"Index":707,"Attempt":0,"Launch Time":1427397533892,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1705,"Index":703,"Attempt":0,"Launch Time":1427397533868,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533892,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":23,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":362270,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1710,"Index":708,"Attempt":0,"Launch Time":1427397533894,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1706,"Index":704,"Attempt":0,"Launch Time":1427397533870,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533894,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":23,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":535855,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1711,"Index":709,"Attempt":0,"Launch Time":1427397533895,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1708,"Index":706,"Attempt":0,"Launch Time":1427397533880,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533895,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":378036,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1712,"Index":710,"Attempt":0,"Launch Time":1427397533896,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1707,"Index":705,"Attempt":0,"Launch Time":1427397533879,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533897,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9615,"Shuffle Write Time":380196,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1713,"Index":711,"Attempt":0,"Launch Time":1427397533897,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1704,"Index":702,"Attempt":0,"Launch Time":1427397533868,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533898,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":26,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":394251,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1714,"Index":712,"Attempt":0,"Launch Time":1427397533899,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1715,"Index":713,"Attempt":0,"Launch Time":1427397533899,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1682,"Index":680,"Attempt":0,"Launch Time":1427397533813,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533899,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":33,"Executor Run Time":49,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":4347703,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1680,"Index":678,"Attempt":0,"Launch Time":1427397533807,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533899,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":91,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":336182,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1716,"Index":714,"Attempt":0,"Launch Time":1427397533904,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1710,"Index":708,"Attempt":0,"Launch Time":1427397533894,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533904,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":383082,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1717,"Index":715,"Attempt":0,"Launch Time":1427397533905,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1712,"Index":710,"Attempt":0,"Launch Time":1427397533896,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533905,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":376540,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1718,"Index":716,"Attempt":0,"Launch Time":1427397533906,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1713,"Index":711,"Attempt":0,"Launch Time":1427397533897,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533906,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":393533,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1719,"Index":717,"Attempt":0,"Launch Time":1427397533906,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1714,"Index":712,"Attempt":0,"Launch Time":1427397533899,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533907,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":372266,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1720,"Index":718,"Attempt":0,"Launch Time":1427397533911,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1709,"Index":707,"Attempt":0,"Launch Time":1427397533892,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533912,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":662437,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1721,"Index":719,"Attempt":0,"Launch Time":1427397533915,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1719,"Index":717,"Attempt":0,"Launch Time":1427397533906,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533916,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":459569,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1722,"Index":720,"Attempt":0,"Launch Time":1427397533919,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1717,"Index":715,"Attempt":0,"Launch Time":1427397533905,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533919,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":4765372,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1723,"Index":721,"Attempt":0,"Launch Time":1427397533920,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1716,"Index":714,"Attempt":0,"Launch Time":1427397533904,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533920,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":370638,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1724,"Index":722,"Attempt":0,"Launch Time":1427397533920,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1720,"Index":718,"Attempt":0,"Launch Time":1427397533911,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533921,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":346825,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1725,"Index":723,"Attempt":0,"Launch Time":1427397533922,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1718,"Index":716,"Attempt":0,"Launch Time":1427397533906,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533922,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1122903,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1726,"Index":724,"Attempt":0,"Launch Time":1427397533923,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1715,"Index":713,"Attempt":0,"Launch Time":1427397533899,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533923,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":24,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9613,"Shuffle Write Time":373998,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1727,"Index":725,"Attempt":0,"Launch Time":1427397533929,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1724,"Index":722,"Attempt":0,"Launch Time":1427397533920,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533929,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":358644,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1728,"Index":726,"Attempt":0,"Launch Time":1427397533929,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1723,"Index":721,"Attempt":0,"Launch Time":1427397533920,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533930,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":388573,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1729,"Index":727,"Attempt":0,"Launch Time":1427397533930,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1725,"Index":723,"Attempt":0,"Launch Time":1427397533922,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533930,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":382960,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1730,"Index":728,"Attempt":0,"Launch Time":1427397533932,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1722,"Index":720,"Attempt":0,"Launch Time":1427397533919,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533932,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":370867,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1731,"Index":729,"Attempt":0,"Launch Time":1427397533946,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1721,"Index":719,"Attempt":0,"Launch Time":1427397533915,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533947,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":30,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":387910,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1732,"Index":730,"Attempt":0,"Launch Time":1427397533958,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1711,"Index":709,"Attempt":0,"Launch Time":1427397533895,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533958,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":392066,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1733,"Index":731,"Attempt":0,"Launch Time":1427397533959,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1734,"Index":732,"Attempt":0,"Launch Time":1427397533959,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1729,"Index":727,"Attempt":0,"Launch Time":1427397533930,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533959,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":23,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":5456710,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1726,"Index":724,"Attempt":0,"Launch Time":1427397533923,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533960,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":30,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":800673,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1735,"Index":733,"Attempt":0,"Launch Time":1427397533960,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1678,"Index":676,"Attempt":0,"Launch Time":1427397533802,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533960,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":150,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":498693,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1736,"Index":734,"Attempt":0,"Launch Time":1427397533969,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1732,"Index":730,"Attempt":0,"Launch Time":1427397533958,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533969,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":393694,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1737,"Index":735,"Attempt":0,"Launch Time":1427397533969,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533978,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1731,"Index":729,"Attempt":0,"Launch Time":1427397533946,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533969,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":7,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":475987,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1738,"Index":736,"Attempt":0,"Launch Time":1427397533970,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533980,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1733,"Index":731,"Attempt":0,"Launch Time":1427397533959,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533970,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":386577,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1739,"Index":737,"Attempt":0,"Launch Time":1427397533971,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533980,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1735,"Index":733,"Attempt":0,"Launch Time":1427397533960,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533971,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":382907,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1740,"Index":738,"Attempt":0,"Launch Time":1427397533972,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533985,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1734,"Index":732,"Attempt":0,"Launch Time":1427397533959,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533972,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":418603,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1741,"Index":739,"Attempt":0,"Launch Time":1427397533977,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533988,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1736,"Index":734,"Attempt":0,"Launch Time":1427397533969,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533977,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":398515,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1742,"Index":740,"Attempt":0,"Launch Time":1427397533978,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1737,"Index":735,"Attempt":0,"Launch Time":1427397533969,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533978,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":401819,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1743,"Index":741,"Attempt":0,"Launch Time":1427397533980,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533988,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1738,"Index":736,"Attempt":0,"Launch Time":1427397533970,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533980,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":430070,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1744,"Index":742,"Attempt":0,"Launch Time":1427397533980,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533988,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1739,"Index":737,"Attempt":0,"Launch Time":1427397533971,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533980,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":378962,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1745,"Index":743,"Attempt":0,"Launch Time":1427397533985,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534008,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1740,"Index":738,"Attempt":0,"Launch Time":1427397533972,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533985,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":375816,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1746,"Index":744,"Attempt":0,"Launch Time":1427397533987,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1747,"Index":745,"Attempt":0,"Launch Time":1427397533988,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1741,"Index":739,"Attempt":0,"Launch Time":1427397533977,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533988,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":785295,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1743,"Index":741,"Attempt":0,"Launch Time":1427397533980,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533988,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":374950,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1748,"Index":746,"Attempt":0,"Launch Time":1427397533988,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534010,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1744,"Index":742,"Attempt":0,"Launch Time":1427397533980,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533988,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":379174,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1749,"Index":747,"Attempt":0,"Launch Time":1427397534006,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534015,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1742,"Index":740,"Attempt":0,"Launch Time":1427397533978,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534006,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":28,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":893073,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1750,"Index":748,"Attempt":0,"Launch Time":1427397534008,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1745,"Index":743,"Attempt":0,"Launch Time":1427397533985,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534008,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":23,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":410379,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1751,"Index":749,"Attempt":0,"Launch Time":1427397534010,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1748,"Index":746,"Attempt":0,"Launch Time":1427397533988,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534010,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":375687,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1752,"Index":750,"Attempt":0,"Launch Time":1427397534015,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1749,"Index":747,"Attempt":0,"Launch Time":1427397534006,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534015,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":389610,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1753,"Index":751,"Attempt":0,"Launch Time":1427397534017,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1750,"Index":748,"Attempt":0,"Launch Time":1427397534008,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534017,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":404996,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1754,"Index":752,"Attempt":0,"Launch Time":1427397534018,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1747,"Index":745,"Attempt":0,"Launch Time":1427397533988,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534018,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":30,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":410396,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1755,"Index":753,"Attempt":0,"Launch Time":1427397534021,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1751,"Index":749,"Attempt":0,"Launch Time":1427397534010,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534021,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":385529,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1756,"Index":754,"Attempt":0,"Launch Time":1427397534026,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1757,"Index":755,"Attempt":0,"Launch Time":1427397534026,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1753,"Index":751,"Attempt":0,"Launch Time":1427397534017,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534026,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":383935,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1754,"Index":752,"Attempt":0,"Launch Time":1427397534018,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534026,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":373789,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1758,"Index":756,"Attempt":0,"Launch Time":1427397534028,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1752,"Index":750,"Attempt":0,"Launch Time":1427397534015,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534028,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":434779,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1759,"Index":757,"Attempt":0,"Launch Time":1427397534030,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1755,"Index":753,"Attempt":0,"Launch Time":1427397534021,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534030,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":376569,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1760,"Index":758,"Attempt":0,"Launch Time":1427397534034,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1756,"Index":754,"Attempt":0,"Launch Time":1427397534026,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534034,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":385808,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1761,"Index":759,"Attempt":0,"Launch Time":1427397534038,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1758,"Index":756,"Attempt":0,"Launch Time":1427397534028,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534038,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":543255,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1762,"Index":760,"Attempt":0,"Launch Time":1427397534040,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1759,"Index":757,"Attempt":0,"Launch Time":1427397534030,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534040,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":400592,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1763,"Index":761,"Attempt":0,"Launch Time":1427397534042,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1760,"Index":758,"Attempt":0,"Launch Time":1427397534034,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534042,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":360346,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1764,"Index":762,"Attempt":0,"Launch Time":1427397534042,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1757,"Index":755,"Attempt":0,"Launch Time":1427397534026,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534043,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":786785,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1765,"Index":763,"Attempt":0,"Launch Time":1427397534046,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1761,"Index":759,"Attempt":0,"Launch Time":1427397534038,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534046,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":362999,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1766,"Index":764,"Attempt":0,"Launch Time":1427397534048,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1762,"Index":760,"Attempt":0,"Launch Time":1427397534040,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534048,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":379936,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1767,"Index":765,"Attempt":0,"Launch Time":1427397534051,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1763,"Index":761,"Attempt":0,"Launch Time":1427397534042,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534051,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":372877,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1768,"Index":766,"Attempt":0,"Launch Time":1427397534054,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1765,"Index":763,"Attempt":0,"Launch Time":1427397534046,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534054,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":376299,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1769,"Index":767,"Attempt":0,"Launch Time":1427397534055,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1764,"Index":762,"Attempt":0,"Launch Time":1427397534042,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534055,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":653560,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1770,"Index":768,"Attempt":0,"Launch Time":1427397534056,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1766,"Index":764,"Attempt":0,"Launch Time":1427397534048,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534056,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":380614,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1771,"Index":769,"Attempt":0,"Launch Time":1427397534058,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1767,"Index":765,"Attempt":0,"Launch Time":1427397534051,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534058,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":367832,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1772,"Index":770,"Attempt":0,"Launch Time":1427397534061,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1730,"Index":728,"Attempt":0,"Launch Time":1427397533932,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534062,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":129,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":425225,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1773,"Index":771,"Attempt":0,"Launch Time":1427397534064,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1728,"Index":726,"Attempt":0,"Launch Time":1427397533929,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534065,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":134,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":2642563,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1774,"Index":772,"Attempt":0,"Launch Time":1427397534065,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1770,"Index":768,"Attempt":0,"Launch Time":1427397534056,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534065,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":424451,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1775,"Index":773,"Attempt":0,"Launch Time":1427397534065,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1727,"Index":725,"Attempt":0,"Launch Time":1427397533929,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534065,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":135,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":441656,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1776,"Index":774,"Attempt":0,"Launch Time":1427397534073,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1772,"Index":770,"Attempt":0,"Launch Time":1427397534061,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534073,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":374395,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1777,"Index":775,"Attempt":0,"Launch Time":1427397534076,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1773,"Index":771,"Attempt":0,"Launch Time":1427397534064,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534076,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":446719,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1778,"Index":776,"Attempt":0,"Launch Time":1427397534077,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1774,"Index":772,"Attempt":0,"Launch Time":1427397534065,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534077,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1397794,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1779,"Index":777,"Attempt":0,"Launch Time":1427397534082,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1746,"Index":744,"Attempt":0,"Launch Time":1427397533987,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534082,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":93,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":466742,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1780,"Index":778,"Attempt":0,"Launch Time":1427397534085,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1776,"Index":774,"Attempt":0,"Launch Time":1427397534073,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534085,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":369745,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1781,"Index":779,"Attempt":0,"Launch Time":1427397534086,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1778,"Index":776,"Attempt":0,"Launch Time":1427397534077,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534086,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":367520,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1782,"Index":780,"Attempt":0,"Launch Time":1427397534086,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1777,"Index":775,"Attempt":0,"Launch Time":1427397534076,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534087,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":386816,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1783,"Index":781,"Attempt":0,"Launch Time":1427397534092,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1780,"Index":778,"Attempt":0,"Launch Time":1427397534085,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534092,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":381397,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1784,"Index":782,"Attempt":0,"Launch Time":1427397534095,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1781,"Index":779,"Attempt":0,"Launch Time":1427397534086,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534096,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":416193,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1785,"Index":783,"Attempt":0,"Launch Time":1427397534102,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1783,"Index":781,"Attempt":0,"Launch Time":1427397534092,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534102,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":447791,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1786,"Index":784,"Attempt":0,"Launch Time":1427397534103,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1787,"Index":785,"Attempt":0,"Launch Time":1427397534103,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1784,"Index":782,"Attempt":0,"Launch Time":1427397534095,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534104,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":353523,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1782,"Index":780,"Attempt":0,"Launch Time":1427397534086,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534104,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":412108,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1788,"Index":786,"Attempt":0,"Launch Time":1427397534107,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1779,"Index":777,"Attempt":0,"Launch Time":1427397534082,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534108,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1132439,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1789,"Index":787,"Attempt":0,"Launch Time":1427397534111,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1785,"Index":783,"Attempt":0,"Launch Time":1427397534102,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534111,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":345063,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1790,"Index":788,"Attempt":0,"Launch Time":1427397534111,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1791,"Index":789,"Attempt":0,"Launch Time":1427397534112,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1787,"Index":785,"Attempt":0,"Launch Time":1427397534103,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534112,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":352142,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1786,"Index":784,"Attempt":0,"Launch Time":1427397534103,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534112,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":348652,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1792,"Index":790,"Attempt":0,"Launch Time":1427397534118,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1788,"Index":786,"Attempt":0,"Launch Time":1427397534107,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534118,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":365875,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1793,"Index":791,"Attempt":0,"Launch Time":1427397534119,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1789,"Index":787,"Attempt":0,"Launch Time":1427397534111,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534119,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":350400,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1794,"Index":792,"Attempt":0,"Launch Time":1427397534119,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1791,"Index":789,"Attempt":0,"Launch Time":1427397534112,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534120,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":350956,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1795,"Index":793,"Attempt":0,"Launch Time":1427397534121,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1790,"Index":788,"Attempt":0,"Launch Time":1427397534111,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534121,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1346837,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1796,"Index":794,"Attempt":0,"Launch Time":1427397534142,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1795,"Index":793,"Attempt":0,"Launch Time":1427397534121,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534143,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":567504,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1797,"Index":795,"Attempt":0,"Launch Time":1427397534146,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1798,"Index":796,"Attempt":0,"Launch Time":1427397534147,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1794,"Index":792,"Attempt":0,"Launch Time":1427397534119,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534147,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":26,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":452054,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1793,"Index":791,"Attempt":0,"Launch Time":1427397534119,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534147,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":27,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":414450,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1799,"Index":797,"Attempt":0,"Launch Time":1427397534152,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1796,"Index":794,"Attempt":0,"Launch Time":1427397534142,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534152,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":357111,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1800,"Index":798,"Attempt":0,"Launch Time":1427397534155,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1797,"Index":795,"Attempt":0,"Launch Time":1427397534146,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534155,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":359351,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1798,"Index":796,"Attempt":0,"Launch Time":1427397534147,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534156,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":370668,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1801,"Index":799,"Attempt":0,"Launch Time":1427397534156,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1802,"Index":800,"Attempt":0,"Launch Time":1427397534161,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1799,"Index":797,"Attempt":0,"Launch Time":1427397534152,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534161,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":348909,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1803,"Index":801,"Attempt":0,"Launch Time":1427397534166,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1800,"Index":798,"Attempt":0,"Launch Time":1427397534155,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534166,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":413427,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1804,"Index":802,"Attempt":0,"Launch Time":1427397534167,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1801,"Index":799,"Attempt":0,"Launch Time":1427397534156,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534167,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":402088,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1805,"Index":803,"Attempt":0,"Launch Time":1427397534169,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1802,"Index":800,"Attempt":0,"Launch Time":1427397534161,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534170,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":400290,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1806,"Index":804,"Attempt":0,"Launch Time":1427397534173,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1803,"Index":801,"Attempt":0,"Launch Time":1427397534166,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534173,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":362576,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1807,"Index":805,"Attempt":0,"Launch Time":1427397534177,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1805,"Index":803,"Attempt":0,"Launch Time":1427397534169,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534177,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":335633,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1808,"Index":806,"Attempt":0,"Launch Time":1427397534180,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1804,"Index":802,"Attempt":0,"Launch Time":1427397534167,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534180,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":387482,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1809,"Index":807,"Attempt":0,"Launch Time":1427397534181,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1806,"Index":804,"Attempt":0,"Launch Time":1427397534173,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534181,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":345430,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1810,"Index":808,"Attempt":0,"Launch Time":1427397534187,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1775,"Index":773,"Attempt":0,"Launch Time":1427397534065,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534187,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":117,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":777772,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1811,"Index":809,"Attempt":0,"Launch Time":1427397534195,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1768,"Index":766,"Attempt":0,"Launch Time":1427397534054,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534195,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":139,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":45244826,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1812,"Index":810,"Attempt":0,"Launch Time":1427397534195,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1771,"Index":769,"Attempt":0,"Launch Time":1427397534058,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534195,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":136,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":862993,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1813,"Index":811,"Attempt":0,"Launch Time":1427397534196,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1792,"Index":790,"Attempt":0,"Launch Time":1427397534118,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534196,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":75,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":403081,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1814,"Index":812,"Attempt":0,"Launch Time":1427397534197,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1810,"Index":808,"Attempt":0,"Launch Time":1427397534187,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534197,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":406893,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1815,"Index":813,"Attempt":0,"Launch Time":1427397534199,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1769,"Index":767,"Attempt":0,"Launch Time":1427397534055,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534199,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":143,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":3810966,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1816,"Index":814,"Attempt":0,"Launch Time":1427397534202,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1811,"Index":809,"Attempt":0,"Launch Time":1427397534195,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534202,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":352776,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1817,"Index":815,"Attempt":0,"Launch Time":1427397534203,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1812,"Index":810,"Attempt":0,"Launch Time":1427397534195,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534203,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":333398,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1818,"Index":816,"Attempt":0,"Launch Time":1427397534205,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1814,"Index":812,"Attempt":0,"Launch Time":1427397534197,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534205,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":467627,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1819,"Index":817,"Attempt":0,"Launch Time":1427397534205,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1813,"Index":811,"Attempt":0,"Launch Time":1427397534196,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534205,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":336095,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1820,"Index":818,"Attempt":0,"Launch Time":1427397534208,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1815,"Index":813,"Attempt":0,"Launch Time":1427397534199,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534208,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":351815,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1821,"Index":819,"Attempt":0,"Launch Time":1427397534210,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1816,"Index":814,"Attempt":0,"Launch Time":1427397534202,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534210,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":319362,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1817,"Index":815,"Attempt":0,"Launch Time":1427397534203,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534212,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":341096,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1822,"Index":820,"Attempt":0,"Launch Time":1427397534212,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1823,"Index":821,"Attempt":0,"Launch Time":1427397534212,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1819,"Index":817,"Attempt":0,"Launch Time":1427397534205,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534212,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":301402,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1824,"Index":822,"Attempt":0,"Launch Time":1427397534219,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1825,"Index":823,"Attempt":0,"Launch Time":1427397534220,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1822,"Index":820,"Attempt":0,"Launch Time":1427397534212,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534220,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":334606,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1818,"Index":816,"Attempt":0,"Launch Time":1427397534205,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534220,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":349297,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1826,"Index":824,"Attempt":0,"Launch Time":1427397534220,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1821,"Index":819,"Attempt":0,"Launch Time":1427397534210,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534220,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":398798,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1827,"Index":825,"Attempt":0,"Launch Time":1427397534224,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1823,"Index":821,"Attempt":0,"Launch Time":1427397534212,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534224,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":438821,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1828,"Index":826,"Attempt":0,"Launch Time":1427397534225,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1820,"Index":818,"Attempt":0,"Launch Time":1427397534208,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534226,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":461853,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1829,"Index":827,"Attempt":0,"Launch Time":1427397534227,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1824,"Index":822,"Attempt":0,"Launch Time":1427397534219,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534227,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":342253,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1830,"Index":828,"Attempt":0,"Launch Time":1427397534227,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1826,"Index":824,"Attempt":0,"Launch Time":1427397534220,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534227,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":335085,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1831,"Index":829,"Attempt":0,"Launch Time":1427397534233,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1825,"Index":823,"Attempt":0,"Launch Time":1427397534220,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534233,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":348417,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1832,"Index":830,"Attempt":0,"Launch Time":1427397534235,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1833,"Index":831,"Attempt":0,"Launch Time":1427397534236,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1829,"Index":827,"Attempt":0,"Launch Time":1427397534227,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534236,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":419490,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1830,"Index":828,"Attempt":0,"Launch Time":1427397534227,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534236,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":401931,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1834,"Index":832,"Attempt":0,"Launch Time":1427397534238,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1828,"Index":826,"Attempt":0,"Launch Time":1427397534225,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534238,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":386199,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1835,"Index":833,"Attempt":0,"Launch Time":1427397534241,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1827,"Index":825,"Attempt":0,"Launch Time":1427397534224,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534241,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":385252,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1836,"Index":834,"Attempt":0,"Launch Time":1427397534242,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1831,"Index":829,"Attempt":0,"Launch Time":1427397534233,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534242,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":401371,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1837,"Index":835,"Attempt":0,"Launch Time":1427397534243,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1832,"Index":830,"Attempt":0,"Launch Time":1427397534235,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534244,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":361497,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1838,"Index":836,"Attempt":0,"Launch Time":1427397534245,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1833,"Index":831,"Attempt":0,"Launch Time":1427397534236,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534245,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":436663,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1839,"Index":837,"Attempt":0,"Launch Time":1427397534258,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1836,"Index":834,"Attempt":0,"Launch Time":1427397534242,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534258,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":474590,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1840,"Index":838,"Attempt":0,"Launch Time":1427397534259,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1841,"Index":839,"Attempt":0,"Launch Time":1427397534259,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1835,"Index":833,"Attempt":0,"Launch Time":1427397534241,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534260,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":313298,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1834,"Index":832,"Attempt":0,"Launch Time":1427397534238,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534260,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":15,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":377903,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1842,"Index":840,"Attempt":0,"Launch Time":1427397534267,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1843,"Index":841,"Attempt":0,"Launch Time":1427397534267,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1840,"Index":838,"Attempt":0,"Launch Time":1427397534259,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534267,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":364609,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1841,"Index":839,"Attempt":0,"Launch Time":1427397534259,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534268,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":348169,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1844,"Index":842,"Attempt":0,"Launch Time":1427397534273,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1839,"Index":837,"Attempt":0,"Launch Time":1427397534258,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534274,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":396341,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1845,"Index":843,"Attempt":0,"Launch Time":1427397534275,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1842,"Index":840,"Attempt":0,"Launch Time":1427397534267,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534275,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":330072,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1846,"Index":844,"Attempt":0,"Launch Time":1427397534281,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1844,"Index":842,"Attempt":0,"Launch Time":1427397534273,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534281,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":739907,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1847,"Index":845,"Attempt":0,"Launch Time":1427397534282,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1843,"Index":841,"Attempt":0,"Launch Time":1427397534267,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534282,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":336175,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1848,"Index":846,"Attempt":0,"Launch Time":1427397534282,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1845,"Index":843,"Attempt":0,"Launch Time":1427397534275,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534282,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":363253,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1849,"Index":847,"Attempt":0,"Launch Time":1427397534290,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1847,"Index":845,"Attempt":0,"Launch Time":1427397534282,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534290,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":380333,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1850,"Index":848,"Attempt":0,"Launch Time":1427397534293,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1848,"Index":846,"Attempt":0,"Launch Time":1427397534282,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534293,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":427163,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1851,"Index":849,"Attempt":0,"Launch Time":1427397534294,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1846,"Index":844,"Attempt":0,"Launch Time":1427397534281,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534295,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":494247,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1852,"Index":850,"Attempt":0,"Launch Time":1427397534300,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1808,"Index":806,"Attempt":0,"Launch Time":1427397534180,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534300,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":120,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":434064,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1853,"Index":851,"Attempt":0,"Launch Time":1427397534301,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1849,"Index":847,"Attempt":0,"Launch Time":1427397534290,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534301,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":363108,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1854,"Index":852,"Attempt":0,"Launch Time":1427397534303,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1850,"Index":848,"Attempt":0,"Launch Time":1427397534293,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534303,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":343646,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1855,"Index":853,"Attempt":0,"Launch Time":1427397534304,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1851,"Index":849,"Attempt":0,"Launch Time":1427397534294,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534304,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":343459,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1856,"Index":854,"Attempt":0,"Launch Time":1427397534308,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1852,"Index":850,"Attempt":0,"Launch Time":1427397534300,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534308,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":335946,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1857,"Index":855,"Attempt":0,"Launch Time":1427397534309,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1853,"Index":851,"Attempt":0,"Launch Time":1427397534301,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534310,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":353862,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1858,"Index":856,"Attempt":0,"Launch Time":1427397534310,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1854,"Index":852,"Attempt":0,"Launch Time":1427397534303,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534310,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":347505,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1859,"Index":857,"Attempt":0,"Launch Time":1427397534313,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1807,"Index":805,"Attempt":0,"Launch Time":1427397534177,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534313,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":135,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":417549,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1860,"Index":858,"Attempt":0,"Launch Time":1427397534321,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1859,"Index":857,"Attempt":0,"Launch Time":1427397534313,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534321,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":389762,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1861,"Index":859,"Attempt":0,"Launch Time":1427397534329,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1809,"Index":807,"Attempt":0,"Launch Time":1427397534181,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534329,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":148,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":388970,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1862,"Index":860,"Attempt":0,"Launch Time":1427397534331,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1860,"Index":858,"Attempt":0,"Launch Time":1427397534321,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534332,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":351883,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1863,"Index":861,"Attempt":0,"Launch Time":1427397534335,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1856,"Index":854,"Attempt":0,"Launch Time":1427397534308,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534335,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":27,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":352589,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1864,"Index":862,"Attempt":0,"Launch Time":1427397534341,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1862,"Index":860,"Attempt":0,"Launch Time":1427397534331,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534341,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":396675,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1865,"Index":863,"Attempt":0,"Launch Time":1427397534342,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1861,"Index":859,"Attempt":0,"Launch Time":1427397534329,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534342,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":621066,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1866,"Index":864,"Attempt":0,"Launch Time":1427397534344,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1863,"Index":861,"Attempt":0,"Launch Time":1427397534335,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534345,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":445369,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1867,"Index":865,"Attempt":0,"Launch Time":1427397534349,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1864,"Index":862,"Attempt":0,"Launch Time":1427397534341,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534349,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":394144,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1868,"Index":866,"Attempt":0,"Launch Time":1427397534350,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1838,"Index":836,"Attempt":0,"Launch Time":1427397534245,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534350,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":105,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":353139,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1869,"Index":867,"Attempt":0,"Launch Time":1427397534350,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1865,"Index":863,"Attempt":0,"Launch Time":1427397534342,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534350,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":341831,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1870,"Index":868,"Attempt":0,"Launch Time":1427397534352,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1866,"Index":864,"Attempt":0,"Launch Time":1427397534344,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534352,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":334635,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1871,"Index":869,"Attempt":0,"Launch Time":1427397534357,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1867,"Index":865,"Attempt":0,"Launch Time":1427397534349,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534357,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":330024,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1872,"Index":870,"Attempt":0,"Launch Time":1427397534358,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1869,"Index":867,"Attempt":0,"Launch Time":1427397534350,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534358,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":355173,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1873,"Index":871,"Attempt":0,"Launch Time":1427397534361,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1870,"Index":868,"Attempt":0,"Launch Time":1427397534352,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534361,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":420870,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1874,"Index":872,"Attempt":0,"Launch Time":1427397534364,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1871,"Index":869,"Attempt":0,"Launch Time":1427397534357,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534364,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":312942,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1875,"Index":873,"Attempt":0,"Launch Time":1427397534367,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1872,"Index":870,"Attempt":0,"Launch Time":1427397534358,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534367,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":425689,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1876,"Index":874,"Attempt":0,"Launch Time":1427397534368,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1868,"Index":866,"Attempt":0,"Launch Time":1427397534350,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534368,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":471193,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1877,"Index":875,"Attempt":0,"Launch Time":1427397534369,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1873,"Index":871,"Attempt":0,"Launch Time":1427397534361,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534369,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":348462,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1878,"Index":876,"Attempt":0,"Launch Time":1427397534372,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1837,"Index":835,"Attempt":0,"Launch Time":1427397534243,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534372,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":112,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":387801,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1879,"Index":877,"Attempt":0,"Launch Time":1427397534373,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1874,"Index":872,"Attempt":0,"Launch Time":1427397534364,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534373,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":354789,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1880,"Index":878,"Attempt":0,"Launch Time":1427397534375,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1875,"Index":873,"Attempt":0,"Launch Time":1427397534367,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534375,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":360864,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1881,"Index":879,"Attempt":0,"Launch Time":1427397534377,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1876,"Index":874,"Attempt":0,"Launch Time":1427397534368,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534377,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":362266,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1882,"Index":880,"Attempt":0,"Launch Time":1427397534380,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1877,"Index":875,"Attempt":0,"Launch Time":1427397534369,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534380,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":355948,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1883,"Index":881,"Attempt":0,"Launch Time":1427397534382,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1879,"Index":877,"Attempt":0,"Launch Time":1427397534373,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534382,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":444666,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1884,"Index":882,"Attempt":0,"Launch Time":1427397534394,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1880,"Index":878,"Attempt":0,"Launch Time":1427397534375,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534395,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":19,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":387253,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1885,"Index":883,"Attempt":0,"Launch Time":1427397534398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1882,"Index":880,"Attempt":0,"Launch Time":1427397534380,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534399,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":405014,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1886,"Index":884,"Attempt":0,"Launch Time":1427397534400,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1881,"Index":879,"Attempt":0,"Launch Time":1427397534377,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534400,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":23,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":445912,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1887,"Index":885,"Attempt":0,"Launch Time":1427397534408,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1886,"Index":884,"Attempt":0,"Launch Time":1427397534400,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534408,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":349924,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1888,"Index":886,"Attempt":0,"Launch Time":1427397534412,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1883,"Index":881,"Attempt":0,"Launch Time":1427397534382,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534412,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":333210,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1889,"Index":887,"Attempt":0,"Launch Time":1427397534412,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1884,"Index":882,"Attempt":0,"Launch Time":1427397534394,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534412,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":815597,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1890,"Index":888,"Attempt":0,"Launch Time":1427397534415,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1878,"Index":876,"Attempt":0,"Launch Time":1427397534372,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534416,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":39,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":8429185,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1891,"Index":889,"Attempt":0,"Launch Time":1427397534418,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1888,"Index":886,"Attempt":0,"Launch Time":1427397534412,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534418,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":301230,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1892,"Index":890,"Attempt":0,"Launch Time":1427397534419,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1889,"Index":887,"Attempt":0,"Launch Time":1427397534412,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534420,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":332304,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1893,"Index":891,"Attempt":0,"Launch Time":1427397534421,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1887,"Index":885,"Attempt":0,"Launch Time":1427397534408,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534421,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":352623,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1894,"Index":892,"Attempt":0,"Launch Time":1427397534422,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1890,"Index":888,"Attempt":0,"Launch Time":1427397534415,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534423,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":334619,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1895,"Index":893,"Attempt":0,"Launch Time":1427397534425,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1885,"Index":883,"Attempt":0,"Launch Time":1427397534398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534425,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":25,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":2945781,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1896,"Index":894,"Attempt":0,"Launch Time":1427397534427,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1891,"Index":889,"Attempt":0,"Launch Time":1427397534418,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534427,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":332908,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1897,"Index":895,"Attempt":0,"Launch Time":1427397534428,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1892,"Index":890,"Attempt":0,"Launch Time":1427397534419,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534428,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":331771,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1898,"Index":896,"Attempt":0,"Launch Time":1427397534429,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1893,"Index":891,"Attempt":0,"Launch Time":1427397534421,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534429,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":316333,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1899,"Index":897,"Attempt":0,"Launch Time":1427397534429,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1894,"Index":892,"Attempt":0,"Launch Time":1427397534422,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534429,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":312546,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1900,"Index":898,"Attempt":0,"Launch Time":1427397534430,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1855,"Index":853,"Attempt":0,"Launch Time":1427397534304,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534431,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":125,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":28161927,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1901,"Index":899,"Attempt":0,"Launch Time":1427397534435,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1897,"Index":895,"Attempt":0,"Launch Time":1427397534428,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534436,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":364433,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1902,"Index":900,"Attempt":0,"Launch Time":1427397534436,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1896,"Index":894,"Attempt":0,"Launch Time":1427397534427,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534436,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":359400,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1903,"Index":901,"Attempt":0,"Launch Time":1427397534437,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1895,"Index":893,"Attempt":0,"Launch Time":1427397534425,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534437,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":371274,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1904,"Index":902,"Attempt":0,"Launch Time":1427397534438,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1858,"Index":856,"Attempt":0,"Launch Time":1427397534310,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534438,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":127,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":354588,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1905,"Index":903,"Attempt":0,"Launch Time":1427397534439,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1898,"Index":896,"Attempt":0,"Launch Time":1427397534429,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534439,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":346907,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1906,"Index":904,"Attempt":0,"Launch Time":1427397534444,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1857,"Index":855,"Attempt":0,"Launch Time":1427397534309,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534444,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":51,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":405777,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1907,"Index":905,"Attempt":0,"Launch Time":1427397534445,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1900,"Index":898,"Attempt":0,"Launch Time":1427397534430,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534445,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":397871,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1908,"Index":906,"Attempt":0,"Launch Time":1427397534449,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1904,"Index":902,"Attempt":0,"Launch Time":1427397534438,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534449,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":585569,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1909,"Index":907,"Attempt":0,"Launch Time":1427397534453,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1906,"Index":904,"Attempt":0,"Launch Time":1427397534444,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534453,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":377628,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1910,"Index":908,"Attempt":0,"Launch Time":1427397534454,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1907,"Index":905,"Attempt":0,"Launch Time":1427397534445,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534454,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":362359,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1911,"Index":909,"Attempt":0,"Launch Time":1427397534459,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1899,"Index":897,"Attempt":0,"Launch Time":1427397534429,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534459,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":30,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":657981,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1912,"Index":910,"Attempt":0,"Launch Time":1427397534460,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1908,"Index":906,"Attempt":0,"Launch Time":1427397534449,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534460,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":394723,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1913,"Index":911,"Attempt":0,"Launch Time":1427397534462,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1909,"Index":907,"Attempt":0,"Launch Time":1427397534453,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534462,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":358204,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1914,"Index":912,"Attempt":0,"Launch Time":1427397534462,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1910,"Index":908,"Attempt":0,"Launch Time":1427397534454,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534462,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":356363,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1915,"Index":913,"Attempt":0,"Launch Time":1427397534469,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1912,"Index":910,"Attempt":0,"Launch Time":1427397534460,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534469,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":348522,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1916,"Index":914,"Attempt":0,"Launch Time":1427397534471,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1913,"Index":911,"Attempt":0,"Launch Time":1427397534462,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534472,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":379800,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1917,"Index":915,"Attempt":0,"Launch Time":1427397534472,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1914,"Index":912,"Attempt":0,"Launch Time":1427397534462,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534472,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":372249,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1918,"Index":916,"Attempt":0,"Launch Time":1427397534475,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1911,"Index":909,"Attempt":0,"Launch Time":1427397534459,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534475,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":367741,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1919,"Index":917,"Attempt":0,"Launch Time":1427397534478,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1915,"Index":913,"Attempt":0,"Launch Time":1427397534469,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534478,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":373089,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1920,"Index":918,"Attempt":0,"Launch Time":1427397534480,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1916,"Index":914,"Attempt":0,"Launch Time":1427397534471,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534481,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":418814,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1921,"Index":919,"Attempt":0,"Launch Time":1427397534481,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1917,"Index":915,"Attempt":0,"Launch Time":1427397534472,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534481,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":489944,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1922,"Index":920,"Attempt":0,"Launch Time":1427397534486,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1918,"Index":916,"Attempt":0,"Launch Time":1427397534475,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534487,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":359340,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1923,"Index":921,"Attempt":0,"Launch Time":1427397534487,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1919,"Index":917,"Attempt":0,"Launch Time":1427397534478,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534487,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":388413,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1924,"Index":922,"Attempt":0,"Launch Time":1427397534489,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1920,"Index":918,"Attempt":0,"Launch Time":1427397534480,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534489,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":334917,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1925,"Index":923,"Attempt":0,"Launch Time":1427397534490,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1921,"Index":919,"Attempt":0,"Launch Time":1427397534481,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534490,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1385840,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1926,"Index":924,"Attempt":0,"Launch Time":1427397534494,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1923,"Index":921,"Attempt":0,"Launch Time":1427397534487,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534494,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":313926,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1927,"Index":925,"Attempt":0,"Launch Time":1427397534496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1922,"Index":920,"Attempt":0,"Launch Time":1427397534486,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534496,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":327943,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1928,"Index":926,"Attempt":0,"Launch Time":1427397534497,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1924,"Index":922,"Attempt":0,"Launch Time":1427397534489,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534497,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":331695,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1929,"Index":927,"Attempt":0,"Launch Time":1427397534499,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1925,"Index":923,"Attempt":0,"Launch Time":1427397534490,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534500,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":501427,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1930,"Index":928,"Attempt":0,"Launch Time":1427397534514,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1926,"Index":924,"Attempt":0,"Launch Time":1427397534494,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534515,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":332346,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1931,"Index":929,"Attempt":0,"Launch Time":1427397534519,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1905,"Index":903,"Attempt":0,"Launch Time":1427397534439,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534519,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":23,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9608,"Shuffle Write Time":353643,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1932,"Index":930,"Attempt":0,"Launch Time":1427397534519,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1927,"Index":925,"Attempt":0,"Launch Time":1427397534496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534519,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":338074,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1933,"Index":931,"Attempt":0,"Launch Time":1427397534522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1934,"Index":932,"Attempt":0,"Launch Time":1427397534523,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1929,"Index":927,"Attempt":0,"Launch Time":1427397534499,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534523,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":19,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":344812,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1928,"Index":926,"Attempt":0,"Launch Time":1427397534497,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534523,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":462711,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1935,"Index":933,"Attempt":0,"Launch Time":1427397534523,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1930,"Index":928,"Attempt":0,"Launch Time":1427397534514,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534523,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":373195,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1936,"Index":934,"Attempt":0,"Launch Time":1427397534529,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1931,"Index":929,"Attempt":0,"Launch Time":1427397534519,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534529,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":465886,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1937,"Index":935,"Attempt":0,"Launch Time":1427397534532,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1932,"Index":930,"Attempt":0,"Launch Time":1427397534519,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534532,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":439219,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1938,"Index":936,"Attempt":0,"Launch Time":1427397534532,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1934,"Index":932,"Attempt":0,"Launch Time":1427397534523,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534532,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":469687,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1939,"Index":937,"Attempt":0,"Launch Time":1427397534533,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1935,"Index":933,"Attempt":0,"Launch Time":1427397534523,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534534,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":448794,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1940,"Index":938,"Attempt":0,"Launch Time":1427397534534,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1903,"Index":901,"Attempt":0,"Launch Time":1427397534437,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534534,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":5,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":454155,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1941,"Index":939,"Attempt":0,"Launch Time":1427397534535,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1933,"Index":931,"Attempt":0,"Launch Time":1427397534522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534535,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":635229,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1942,"Index":940,"Attempt":0,"Launch Time":1427397534536,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1936,"Index":934,"Attempt":0,"Launch Time":1427397534529,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534536,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":336724,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1943,"Index":941,"Attempt":0,"Launch Time":1427397534542,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1937,"Index":935,"Attempt":0,"Launch Time":1427397534532,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534542,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":399594,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1944,"Index":942,"Attempt":0,"Launch Time":1427397534543,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1941,"Index":939,"Attempt":0,"Launch Time":1427397534535,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534543,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":361707,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1945,"Index":943,"Attempt":0,"Launch Time":1427397534544,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1942,"Index":940,"Attempt":0,"Launch Time":1427397534536,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534544,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":326515,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1946,"Index":944,"Attempt":0,"Launch Time":1427397534546,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1940,"Index":938,"Attempt":0,"Launch Time":1427397534534,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534546,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":356563,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1947,"Index":945,"Attempt":0,"Launch Time":1427397534546,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1948,"Index":946,"Attempt":0,"Launch Time":1427397534547,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1939,"Index":937,"Attempt":0,"Launch Time":1427397534533,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534547,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":336760,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1938,"Index":936,"Attempt":0,"Launch Time":1427397534532,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534547,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":855395,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1949,"Index":947,"Attempt":0,"Launch Time":1427397534552,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1945,"Index":943,"Attempt":0,"Launch Time":1427397534544,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534552,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":422436,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1950,"Index":948,"Attempt":0,"Launch Time":1427397534555,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1946,"Index":944,"Attempt":0,"Launch Time":1427397534546,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534556,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":340968,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1951,"Index":949,"Attempt":0,"Launch Time":1427397534556,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1948,"Index":946,"Attempt":0,"Launch Time":1427397534547,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534556,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":325854,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1952,"Index":950,"Attempt":0,"Launch Time":1427397534557,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1944,"Index":942,"Attempt":0,"Launch Time":1427397534543,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534557,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":374871,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1953,"Index":951,"Attempt":0,"Launch Time":1427397534557,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1902,"Index":900,"Attempt":0,"Launch Time":1427397534436,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534558,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":42,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":337912,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1954,"Index":952,"Attempt":0,"Launch Time":1427397534559,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1943,"Index":941,"Attempt":0,"Launch Time":1427397534542,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534559,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":379081,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1955,"Index":953,"Attempt":0,"Launch Time":1427397534560,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1949,"Index":947,"Attempt":0,"Launch Time":1427397534552,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534560,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":339239,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1956,"Index":954,"Attempt":0,"Launch Time":1427397534563,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1947,"Index":945,"Attempt":0,"Launch Time":1427397534546,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534563,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":326052,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1957,"Index":955,"Attempt":0,"Launch Time":1427397534566,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1901,"Index":899,"Attempt":0,"Launch Time":1427397534435,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534566,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":13,"Executor Run Time":42,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":4236414,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1958,"Index":956,"Attempt":0,"Launch Time":1427397534567,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1955,"Index":953,"Attempt":0,"Launch Time":1427397534560,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534567,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":331521,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1959,"Index":957,"Attempt":0,"Launch Time":1427397534575,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1958,"Index":956,"Attempt":0,"Launch Time":1427397534567,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534575,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":383290,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1960,"Index":958,"Attempt":0,"Launch Time":1427397534575,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1957,"Index":955,"Attempt":0,"Launch Time":1427397534566,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534576,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":383532,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1961,"Index":959,"Attempt":0,"Launch Time":1427397534581,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1951,"Index":949,"Attempt":0,"Launch Time":1427397534556,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534581,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":13,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":430177,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1962,"Index":960,"Attempt":0,"Launch Time":1427397534583,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1959,"Index":957,"Attempt":0,"Launch Time":1427397534575,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534583,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":392041,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1963,"Index":961,"Attempt":0,"Launch Time":1427397534584,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1953,"Index":951,"Attempt":0,"Launch Time":1427397534557,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534585,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":458344,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1964,"Index":962,"Attempt":0,"Launch Time":1427397534585,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1960,"Index":958,"Attempt":0,"Launch Time":1427397534575,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534585,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":383063,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1965,"Index":963,"Attempt":0,"Launch Time":1427397534591,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1966,"Index":964,"Attempt":0,"Launch Time":1427397534591,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1962,"Index":960,"Attempt":0,"Launch Time":1427397534583,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534592,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":350436,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1961,"Index":959,"Attempt":0,"Launch Time":1427397534581,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534592,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":337715,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1967,"Index":965,"Attempt":0,"Launch Time":1427397534592,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1964,"Index":962,"Attempt":0,"Launch Time":1427397534585,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534593,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":331616,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1968,"Index":966,"Attempt":0,"Launch Time":1427397534594,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1963,"Index":961,"Attempt":0,"Launch Time":1427397534584,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534595,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":348728,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1969,"Index":967,"Attempt":0,"Launch Time":1427397534599,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1966,"Index":964,"Attempt":0,"Launch Time":1427397534591,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534600,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":340757,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1970,"Index":968,"Attempt":0,"Launch Time":1427397534600,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1967,"Index":965,"Attempt":0,"Launch Time":1427397534592,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534600,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":338434,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1971,"Index":969,"Attempt":0,"Launch Time":1427397534601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1965,"Index":963,"Attempt":0,"Launch Time":1427397534591,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534601,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":372676,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1972,"Index":970,"Attempt":0,"Launch Time":1427397534602,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1968,"Index":966,"Attempt":0,"Launch Time":1427397534594,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534603,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":361503,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1973,"Index":971,"Attempt":0,"Launch Time":1427397534625,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1969,"Index":967,"Attempt":0,"Launch Time":1427397534599,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534626,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":25,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":393111,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1974,"Index":972,"Attempt":0,"Launch Time":1427397534628,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1970,"Index":968,"Attempt":0,"Launch Time":1427397534600,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534628,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":25,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":393008,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1975,"Index":973,"Attempt":0,"Launch Time":1427397534628,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1971,"Index":969,"Attempt":0,"Launch Time":1427397534601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534628,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":26,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":421556,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1976,"Index":974,"Attempt":0,"Launch Time":1427397534635,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1973,"Index":971,"Attempt":0,"Launch Time":1427397534625,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534635,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":349530,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1977,"Index":975,"Attempt":0,"Launch Time":1427397534635,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1974,"Index":972,"Attempt":0,"Launch Time":1427397534628,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534635,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":332727,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1978,"Index":976,"Attempt":0,"Launch Time":1427397534636,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1975,"Index":973,"Attempt":0,"Launch Time":1427397534628,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534636,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":335048,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1979,"Index":977,"Attempt":0,"Launch Time":1427397534638,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1972,"Index":970,"Attempt":0,"Launch Time":1427397534602,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534638,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":34,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":330062,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1980,"Index":978,"Attempt":0,"Launch Time":1427397534639,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1956,"Index":954,"Attempt":0,"Launch Time":1427397534563,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534639,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":314348,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1981,"Index":979,"Attempt":0,"Launch Time":1427397534643,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1976,"Index":974,"Attempt":0,"Launch Time":1427397534635,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534643,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":704497,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1982,"Index":980,"Attempt":0,"Launch Time":1427397534644,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1977,"Index":975,"Attempt":0,"Launch Time":1427397534635,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534644,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":330702,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1983,"Index":981,"Attempt":0,"Launch Time":1427397534647,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1980,"Index":978,"Attempt":0,"Launch Time":1427397534639,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534647,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":321206,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1984,"Index":982,"Attempt":0,"Launch Time":1427397534647,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1979,"Index":977,"Attempt":0,"Launch Time":1427397534638,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534647,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":329554,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1985,"Index":983,"Attempt":0,"Launch Time":1427397534648,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1978,"Index":976,"Attempt":0,"Launch Time":1427397534636,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534648,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":332707,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1986,"Index":984,"Attempt":0,"Launch Time":1427397534651,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1982,"Index":980,"Attempt":0,"Launch Time":1427397534644,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534651,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":350326,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1987,"Index":985,"Attempt":0,"Launch Time":1427397534652,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1981,"Index":979,"Attempt":0,"Launch Time":1427397534643,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534652,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":410055,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1988,"Index":986,"Attempt":0,"Launch Time":1427397534656,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1985,"Index":983,"Attempt":0,"Launch Time":1427397534648,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534656,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":364294,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1989,"Index":987,"Attempt":0,"Launch Time":1427397534659,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1984,"Index":982,"Attempt":0,"Launch Time":1427397534647,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534659,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":608205,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1990,"Index":988,"Attempt":0,"Launch Time":1427397534659,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1991,"Index":989,"Attempt":0,"Launch Time":1427397534660,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1987,"Index":985,"Attempt":0,"Launch Time":1427397534652,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534660,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":452351,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1983,"Index":981,"Attempt":0,"Launch Time":1427397534647,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534660,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":389688,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1992,"Index":990,"Attempt":0,"Launch Time":1427397534663,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1986,"Index":984,"Attempt":0,"Launch Time":1427397534651,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534663,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":359581,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1993,"Index":991,"Attempt":0,"Launch Time":1427397534664,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1988,"Index":986,"Attempt":0,"Launch Time":1427397534656,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534665,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":363846,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1994,"Index":992,"Attempt":0,"Launch Time":1427397534665,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1950,"Index":948,"Attempt":0,"Launch Time":1427397534555,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534666,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":108,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1767356,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1995,"Index":993,"Attempt":0,"Launch Time":1427397534667,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1990,"Index":988,"Attempt":0,"Launch Time":1427397534659,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534668,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":379662,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1996,"Index":994,"Attempt":0,"Launch Time":1427397534668,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1991,"Index":989,"Attempt":0,"Launch Time":1427397534660,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534668,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":520453,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1997,"Index":995,"Attempt":0,"Launch Time":1427397534672,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1998,"Index":996,"Attempt":0,"Launch Time":1427397534672,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1992,"Index":990,"Attempt":0,"Launch Time":1427397534663,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534673,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":368161,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1993,"Index":991,"Attempt":0,"Launch Time":1427397534664,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534674,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":368150,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1999,"Index":997,"Attempt":0,"Launch Time":1427397534676,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":2000,"Index":998,"Attempt":0,"Launch Time":1427397534677,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1994,"Index":992,"Attempt":0,"Launch Time":1427397534665,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534678,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":341815,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1996,"Index":994,"Attempt":0,"Launch Time":1427397534668,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534678,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":358850,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":2001,"Index":999,"Attempt":0,"Launch Time":1427397534684,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1989,"Index":987,"Attempt":0,"Launch Time":1427397534659,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534684,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":24,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":404274,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1954,"Index":952,"Attempt":0,"Launch Time":1427397534559,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534684,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":125,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":595541,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1997,"Index":995,"Attempt":0,"Launch Time":1427397534672,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534685,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":377006,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1998,"Index":996,"Attempt":0,"Launch Time":1427397534672,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534687,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":343775,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1952,"Index":950,"Attempt":0,"Launch Time":1427397534557,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534690,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":131,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":2918877,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2001,"Index":999,"Attempt":0,"Launch Time":1427397534684,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534692,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":377136,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2000,"Index":998,"Attempt":0,"Launch Time":1427397534677,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534696,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":325000,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1999,"Index":997,"Attempt":0,"Launch Time":1427397534676,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534701,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":24,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":537520,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1995,"Index":993,"Attempt":0,"Launch Time":1427397534667,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534702,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":9,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":329735,"Shuffle Records Written":100}}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":4,"Stage Attempt ID":0,"Stage Name":"map at :21","Number of Tasks":1000,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:287)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:21)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:28)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:30)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.(:32)\n$line15.$read$$iwC$$iwC$$iwC.(:34)\n$line15.$read$$iwC$$iwC.(:36)\n$line15.$read$$iwC.(:38)\n$line15.$read.(:40)\n$line15.$read$.(:44)\n$line15.$read$.()\n$line15.$eval$.(:7)\n$line15.$eval$.()\n$line15.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Submission Time":1427397532127,"Completion Time":1427397534702,"Accumulables":[]}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":5,"Stage Attempt ID":0,"Stage Name":"next at :26","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":50,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"scala.collection.Iterator$$anon$13.next(Iterator.scala:372)\n$line24.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line24.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:31)\n$line24.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:33)\n$line24.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:35)\n$line24.$read$$iwC$$iwC$$iwC$$iwC.(:37)\n$line24.$read$$iwC$$iwC$$iwC.(:39)\n$line24.$read$$iwC$$iwC.(:41)\n$line24.$read$$iwC.(:43)\n$line24.$read.(:45)\n$line24.$read$.(:49)\n$line24.$read$.()\n$line24.$eval$.(:7)\n$line24.$eval$.()\n$line24.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":2002,"Index":0,"Attempt":0,"Launch Time":1427397534707,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2002,"Index":0,"Attempt":0,"Launch Time":1427397534707,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534876,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":164,"Result Size":1060,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":1000,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":192025,"Total Records Read":2000}}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":5,"Stage Attempt ID":0,"Stage Name":"next at :26","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":50,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"scala.collection.Iterator$$anon$13.next(Iterator.scala:372)\n$line24.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line24.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:31)\n$line24.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:33)\n$line24.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:35)\n$line24.$read$$iwC$$iwC$$iwC$$iwC.(:37)\n$line24.$read$$iwC$$iwC$$iwC.(:39)\n$line24.$read$$iwC$$iwC.(:41)\n$line24.$read$$iwC.(:43)\n$line24.$read.(:45)\n$line24.$read$.(:49)\n$line24.$read$.()\n$line24.$eval$.(:7)\n$line24.$eval$.()\n$line24.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Submission Time":1427397534707,"Completion Time":1427397534876,"Accumulables":[]}} -{"Event":"SparkListenerJobEnd","Job ID":2,"Completion Time":1427397534876,"Job Result":{"Result":"JobSucceeded"}} -{"Event":"SparkListenerJobStart","Job ID":3,"Submission Time":1427397541484,"Stage Infos":[{"Stage ID":6,"Stage Attempt ID":0,"Stage Name":"map at :21","Number of Tasks":1000,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:287)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:21)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:28)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:30)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.(:32)\n$line15.$read$$iwC$$iwC$$iwC.(:34)\n$line15.$read$$iwC$$iwC.(:36)\n$line15.$read$$iwC.(:38)\n$line15.$read.(:40)\n$line15.$read$.(:44)\n$line15.$read$.()\n$line15.$eval$.(:7)\n$line15.$eval$.()\n$line15.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]},{"Stage ID":7,"Stage Attempt ID":0,"Stage Name":"next at :26","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":50,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"scala.collection.Iterator$$anon$13.next(Iterator.scala:372)\n$line26.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line26.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:31)\n$line26.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:33)\n$line26.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:35)\n$line26.$read$$iwC$$iwC$$iwC$$iwC.(:37)\n$line26.$read$$iwC$$iwC$$iwC.(:39)\n$line26.$read$$iwC$$iwC.(:41)\n$line26.$read$$iwC.(:43)\n$line26.$read.(:45)\n$line26.$read$.(:49)\n$line26.$read$.()\n$line26.$eval$.(:7)\n$line26.$eval$.()\n$line26.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]}],"Stage IDs":[6,7]} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":6,"Stage Attempt ID":0,"Stage Name":"map at :21","Number of Tasks":1000,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:287)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:21)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:28)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:30)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.(:32)\n$line15.$read$$iwC$$iwC$$iwC.(:34)\n$line15.$read$$iwC$$iwC.(:36)\n$line15.$read$$iwC.(:38)\n$line15.$read.(:40)\n$line15.$read$.(:44)\n$line15.$read$.()\n$line15.$eval$.(:7)\n$line15.$eval$.()\n$line15.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2003,"Index":0,"Attempt":0,"Launch Time":1427397541495,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2004,"Index":1,"Attempt":0,"Launch Time":1427397541495,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2005,"Index":2,"Attempt":0,"Launch Time":1427397541495,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2006,"Index":3,"Attempt":0,"Launch Time":1427397541495,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2007,"Index":4,"Attempt":0,"Launch Time":1427397541495,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2008,"Index":5,"Attempt":0,"Launch Time":1427397541495,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2009,"Index":6,"Attempt":0,"Launch Time":1427397541495,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2010,"Index":7,"Attempt":0,"Launch Time":1427397541495,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2011,"Index":8,"Attempt":0,"Launch Time":1427397541504,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2009,"Index":6,"Attempt":0,"Launch Time":1427397541495,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541505,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":288694,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2012,"Index":9,"Attempt":0,"Launch Time":1427397541506,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2013,"Index":10,"Attempt":0,"Launch Time":1427397541506,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2003,"Index":0,"Attempt":0,"Launch Time":1427397541495,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541507,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":377403,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2004,"Index":1,"Attempt":0,"Launch Time":1427397541495,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541507,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":323907,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2014,"Index":11,"Attempt":0,"Launch Time":1427397541508,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2015,"Index":12,"Attempt":0,"Launch Time":1427397541509,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2016,"Index":13,"Attempt":0,"Launch Time":1427397541509,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2005,"Index":2,"Attempt":0,"Launch Time":1427397541495,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541509,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":325092,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2006,"Index":3,"Attempt":0,"Launch Time":1427397541495,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541509,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":292197,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2017,"Index":14,"Attempt":0,"Launch Time":1427397541510,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2018,"Index":15,"Attempt":0,"Launch Time":1427397541510,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2007,"Index":4,"Attempt":0,"Launch Time":1427397541495,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541510,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":286278,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2010,"Index":7,"Attempt":0,"Launch Time":1427397541495,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541511,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":651252,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2019,"Index":16,"Attempt":0,"Launch Time":1427397541515,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2020,"Index":17,"Attempt":0,"Launch Time":1427397541516,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2021,"Index":18,"Attempt":0,"Launch Time":1427397541516,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2022,"Index":19,"Attempt":0,"Launch Time":1427397541516,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2023,"Index":20,"Attempt":0,"Launch Time":1427397541516,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2012,"Index":9,"Attempt":0,"Launch Time":1427397541506,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541516,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":320810,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2015,"Index":12,"Attempt":0,"Launch Time":1427397541509,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541516,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":321937,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2013,"Index":10,"Attempt":0,"Launch Time":1427397541506,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541516,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":334729,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2008,"Index":5,"Attempt":0,"Launch Time":1427397541495,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541516,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":307124,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2024,"Index":21,"Attempt":0,"Launch Time":1427397541516,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2014,"Index":11,"Attempt":0,"Launch Time":1427397541508,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541517,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":331453,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2016,"Index":13,"Attempt":0,"Launch Time":1427397541509,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541517,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":314609,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2011,"Index":8,"Attempt":0,"Launch Time":1427397541504,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541518,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":277006,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2025,"Index":22,"Attempt":0,"Launch Time":1427397541524,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2026,"Index":23,"Attempt":0,"Launch Time":1427397541524,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2027,"Index":24,"Attempt":0,"Launch Time":1427397541524,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2028,"Index":25,"Attempt":0,"Launch Time":1427397541524,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2029,"Index":26,"Attempt":0,"Launch Time":1427397541525,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2030,"Index":27,"Attempt":0,"Launch Time":1427397541525,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2031,"Index":28,"Attempt":0,"Launch Time":1427397541525,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2017,"Index":14,"Attempt":0,"Launch Time":1427397541510,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541525,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":272820,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2021,"Index":18,"Attempt":0,"Launch Time":1427397541516,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541526,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":276026,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2018,"Index":15,"Attempt":0,"Launch Time":1427397541510,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541526,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":331530,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2022,"Index":19,"Attempt":0,"Launch Time":1427397541516,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541526,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":279383,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2023,"Index":20,"Attempt":0,"Launch Time":1427397541516,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541526,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":290842,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2019,"Index":16,"Attempt":0,"Launch Time":1427397541515,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541527,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":251904,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2024,"Index":21,"Attempt":0,"Launch Time":1427397541516,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541530,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":284468,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2032,"Index":29,"Attempt":0,"Launch Time":1427397541531,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2033,"Index":30,"Attempt":0,"Launch Time":1427397541531,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2034,"Index":31,"Attempt":0,"Launch Time":1427397541531,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2026,"Index":23,"Attempt":0,"Launch Time":1427397541524,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541532,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":340640,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2020,"Index":17,"Attempt":0,"Launch Time":1427397541516,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541532,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":343566,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2035,"Index":32,"Attempt":0,"Launch Time":1427397541532,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2036,"Index":33,"Attempt":0,"Launch Time":1427397541533,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2027,"Index":24,"Attempt":0,"Launch Time":1427397541524,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541533,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":319003,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2025,"Index":22,"Attempt":0,"Launch Time":1427397541524,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541533,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":329623,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2037,"Index":34,"Attempt":0,"Launch Time":1427397541534,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2038,"Index":35,"Attempt":0,"Launch Time":1427397541534,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2028,"Index":25,"Attempt":0,"Launch Time":1427397541524,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541534,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":364521,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2031,"Index":28,"Attempt":0,"Launch Time":1427397541525,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541534,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":287725,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2039,"Index":36,"Attempt":0,"Launch Time":1427397541534,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2029,"Index":26,"Attempt":0,"Launch Time":1427397541525,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541535,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":291689,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2030,"Index":27,"Attempt":0,"Launch Time":1427397541525,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541536,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":265378,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2040,"Index":37,"Attempt":0,"Launch Time":1427397541538,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2041,"Index":38,"Attempt":0,"Launch Time":1427397541539,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2033,"Index":30,"Attempt":0,"Launch Time":1427397541531,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541539,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":284382,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2042,"Index":39,"Attempt":0,"Launch Time":1427397541539,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2043,"Index":40,"Attempt":0,"Launch Time":1427397541539,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2034,"Index":31,"Attempt":0,"Launch Time":1427397541531,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541539,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":288716,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2036,"Index":33,"Attempt":0,"Launch Time":1427397541533,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541539,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":289000,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2032,"Index":29,"Attempt":0,"Launch Time":1427397541531,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541539,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":273262,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2044,"Index":41,"Attempt":0,"Launch Time":1427397541540,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2045,"Index":42,"Attempt":0,"Launch Time":1427397541540,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2037,"Index":34,"Attempt":0,"Launch Time":1427397541534,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541541,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":280055,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2035,"Index":32,"Attempt":0,"Launch Time":1427397541532,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541541,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":268880,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2046,"Index":43,"Attempt":0,"Launch Time":1427397541544,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2047,"Index":44,"Attempt":0,"Launch Time":1427397541545,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2038,"Index":35,"Attempt":0,"Launch Time":1427397541534,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541546,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":311854,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2048,"Index":45,"Attempt":0,"Launch Time":1427397541547,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2049,"Index":46,"Attempt":0,"Launch Time":1427397541555,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2050,"Index":47,"Attempt":0,"Launch Time":1427397541555,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2051,"Index":48,"Attempt":0,"Launch Time":1427397541555,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2041,"Index":38,"Attempt":0,"Launch Time":1427397541539,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541556,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":351529,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2043,"Index":40,"Attempt":0,"Launch Time":1427397541539,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541556,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":339115,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2042,"Index":39,"Attempt":0,"Launch Time":1427397541539,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541557,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":14,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":393226,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2039,"Index":36,"Attempt":0,"Launch Time":1427397541534,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541557,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":358993,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2040,"Index":37,"Attempt":0,"Launch Time":1427397541538,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541558,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":365613,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2052,"Index":49,"Attempt":0,"Launch Time":1427397541565,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2053,"Index":50,"Attempt":0,"Launch Time":1427397541565,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2054,"Index":51,"Attempt":0,"Launch Time":1427397541565,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2055,"Index":52,"Attempt":0,"Launch Time":1427397541566,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2056,"Index":53,"Attempt":0,"Launch Time":1427397541566,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2049,"Index":46,"Attempt":0,"Launch Time":1427397541555,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541566,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":278951,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2050,"Index":47,"Attempt":0,"Launch Time":1427397541555,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541566,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":248392,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2044,"Index":41,"Attempt":0,"Launch Time":1427397541540,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541569,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":16,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":560782,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2045,"Index":42,"Attempt":0,"Launch Time":1427397541540,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541569,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":18,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":399312,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2057,"Index":54,"Attempt":0,"Launch Time":1427397541570,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2058,"Index":55,"Attempt":0,"Launch Time":1427397541570,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2047,"Index":44,"Attempt":0,"Launch Time":1427397541545,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541570,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":421223,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2046,"Index":43,"Attempt":0,"Launch Time":1427397541544,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541570,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":19,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":260547,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2059,"Index":56,"Attempt":0,"Launch Time":1427397541571,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2048,"Index":45,"Attempt":0,"Launch Time":1427397541547,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541571,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":287921,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2051,"Index":48,"Attempt":0,"Launch Time":1427397541555,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541572,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":337502,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2060,"Index":57,"Attempt":0,"Launch Time":1427397541573,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2061,"Index":58,"Attempt":0,"Launch Time":1427397541573,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2053,"Index":50,"Attempt":0,"Launch Time":1427397541565,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541573,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":306360,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2052,"Index":49,"Attempt":0,"Launch Time":1427397541565,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541574,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":285172,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2054,"Index":51,"Attempt":0,"Launch Time":1427397541565,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541574,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":308404,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2062,"Index":59,"Attempt":0,"Launch Time":1427397541575,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2063,"Index":60,"Attempt":0,"Launch Time":1427397541575,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2064,"Index":61,"Attempt":0,"Launch Time":1427397541575,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2056,"Index":53,"Attempt":0,"Launch Time":1427397541566,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541575,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":284677,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2055,"Index":52,"Attempt":0,"Launch Time":1427397541566,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541576,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":286274,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2065,"Index":62,"Attempt":0,"Launch Time":1427397541577,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2057,"Index":54,"Attempt":0,"Launch Time":1427397541570,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541578,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":333918,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2066,"Index":63,"Attempt":0,"Launch Time":1427397541579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2059,"Index":56,"Attempt":0,"Launch Time":1427397541571,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541580,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":337549,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2067,"Index":64,"Attempt":0,"Launch Time":1427397541580,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2058,"Index":55,"Attempt":0,"Launch Time":1427397541570,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541581,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":431716,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2068,"Index":65,"Attempt":0,"Launch Time":1427397541582,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2069,"Index":66,"Attempt":0,"Launch Time":1427397541582,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2070,"Index":67,"Attempt":0,"Launch Time":1427397541583,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2071,"Index":68,"Attempt":0,"Launch Time":1427397541583,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2060,"Index":57,"Attempt":0,"Launch Time":1427397541573,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541583,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":370201,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2062,"Index":59,"Attempt":0,"Launch Time":1427397541575,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541583,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9610,"Shuffle Write Time":340081,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2064,"Index":61,"Attempt":0,"Launch Time":1427397541575,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541583,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":328851,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2061,"Index":58,"Attempt":0,"Launch Time":1427397541573,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541585,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":386734,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2072,"Index":69,"Attempt":0,"Launch Time":1427397541585,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2063,"Index":60,"Attempt":0,"Launch Time":1427397541575,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541585,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":321268,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2073,"Index":70,"Attempt":0,"Launch Time":1427397541589,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2068,"Index":65,"Attempt":0,"Launch Time":1427397541582,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541589,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":289303,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2074,"Index":71,"Attempt":0,"Launch Time":1427397541590,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2075,"Index":72,"Attempt":0,"Launch Time":1427397541591,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2076,"Index":73,"Attempt":0,"Launch Time":1427397541591,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2066,"Index":63,"Attempt":0,"Launch Time":1427397541579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541591,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":278341,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2069,"Index":66,"Attempt":0,"Launch Time":1427397541582,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541591,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":286191,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2070,"Index":67,"Attempt":0,"Launch Time":1427397541583,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541592,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9614,"Shuffle Write Time":276864,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2077,"Index":74,"Attempt":0,"Launch Time":1427397541592,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2067,"Index":64,"Attempt":0,"Launch Time":1427397541580,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541592,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":282356,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2065,"Index":62,"Attempt":0,"Launch Time":1427397541577,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541592,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":291411,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2078,"Index":75,"Attempt":0,"Launch Time":1427397541593,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2079,"Index":76,"Attempt":0,"Launch Time":1427397541595,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2071,"Index":68,"Attempt":0,"Launch Time":1427397541583,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541595,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":317700,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2080,"Index":77,"Attempt":0,"Launch Time":1427397541597,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2073,"Index":70,"Attempt":0,"Launch Time":1427397541589,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541597,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":352685,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2081,"Index":78,"Attempt":0,"Launch Time":1427397541600,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2072,"Index":69,"Attempt":0,"Launch Time":1427397541585,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541600,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":1048143,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2082,"Index":79,"Attempt":0,"Launch Time":1427397541601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2078,"Index":75,"Attempt":0,"Launch Time":1427397541593,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541601,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":344836,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2083,"Index":80,"Attempt":0,"Launch Time":1427397541603,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2075,"Index":72,"Attempt":0,"Launch Time":1427397541591,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541603,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":308018,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2084,"Index":81,"Attempt":0,"Launch Time":1427397541606,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2080,"Index":77,"Attempt":0,"Launch Time":1427397541597,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541606,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":293481,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2085,"Index":82,"Attempt":0,"Launch Time":1427397541607,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2082,"Index":79,"Attempt":0,"Launch Time":1427397541601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541607,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":295670,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2086,"Index":83,"Attempt":0,"Launch Time":1427397541607,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2081,"Index":78,"Attempt":0,"Launch Time":1427397541600,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541607,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":333637,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2087,"Index":84,"Attempt":0,"Launch Time":1427397541613,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2085,"Index":82,"Attempt":0,"Launch Time":1427397541607,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541613,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":288265,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2088,"Index":85,"Attempt":0,"Launch Time":1427397541613,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2086,"Index":83,"Attempt":0,"Launch Time":1427397541607,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541613,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":287678,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2089,"Index":86,"Attempt":0,"Launch Time":1427397541619,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2088,"Index":85,"Attempt":0,"Launch Time":1427397541613,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541620,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":291447,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2090,"Index":87,"Attempt":0,"Launch Time":1427397541620,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2083,"Index":80,"Attempt":0,"Launch Time":1427397541603,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541620,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":297282,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2091,"Index":88,"Attempt":0,"Launch Time":1427397541620,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2087,"Index":84,"Attempt":0,"Launch Time":1427397541613,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541620,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":410155,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2092,"Index":89,"Attempt":0,"Launch Time":1427397541622,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2076,"Index":73,"Attempt":0,"Launch Time":1427397541591,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541622,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":28,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":298620,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2093,"Index":90,"Attempt":0,"Launch Time":1427397541622,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2084,"Index":81,"Attempt":0,"Launch Time":1427397541606,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541623,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":5,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":315133,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2094,"Index":91,"Attempt":0,"Launch Time":1427397541632,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2090,"Index":87,"Attempt":0,"Launch Time":1427397541620,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541632,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":328543,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2095,"Index":92,"Attempt":0,"Launch Time":1427397541636,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2089,"Index":86,"Attempt":0,"Launch Time":1427397541619,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541636,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":307488,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2096,"Index":93,"Attempt":0,"Launch Time":1427397541636,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2097,"Index":94,"Attempt":0,"Launch Time":1427397541637,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2092,"Index":89,"Attempt":0,"Launch Time":1427397541622,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541637,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":301097,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2079,"Index":76,"Attempt":0,"Launch Time":1427397541595,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541637,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":17,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":290927,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2098,"Index":95,"Attempt":0,"Launch Time":1427397541637,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2093,"Index":90,"Attempt":0,"Launch Time":1427397541622,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541637,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":552776,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2099,"Index":96,"Attempt":0,"Launch Time":1427397541641,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2077,"Index":74,"Attempt":0,"Launch Time":1427397541592,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541641,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":47,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":346052,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2100,"Index":97,"Attempt":0,"Launch Time":1427397541641,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2074,"Index":71,"Attempt":0,"Launch Time":1427397541590,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541641,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":48,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":8493928,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2101,"Index":98,"Attempt":0,"Launch Time":1427397541643,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2095,"Index":92,"Attempt":0,"Launch Time":1427397541636,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541643,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":376515,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2102,"Index":99,"Attempt":0,"Launch Time":1427397541643,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2097,"Index":94,"Attempt":0,"Launch Time":1427397541637,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541644,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":284406,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2103,"Index":100,"Attempt":0,"Launch Time":1427397541644,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2098,"Index":95,"Attempt":0,"Launch Time":1427397541637,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541645,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":310896,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2104,"Index":101,"Attempt":0,"Launch Time":1427397541645,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2094,"Index":91,"Attempt":0,"Launch Time":1427397541632,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541645,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":5541151,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2105,"Index":102,"Attempt":0,"Launch Time":1427397541646,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2091,"Index":88,"Attempt":0,"Launch Time":1427397541620,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541646,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":25,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":655760,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2106,"Index":103,"Attempt":0,"Launch Time":1427397541650,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2101,"Index":98,"Attempt":0,"Launch Time":1427397541643,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541650,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":347454,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2107,"Index":104,"Attempt":0,"Launch Time":1427397541652,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2104,"Index":101,"Attempt":0,"Launch Time":1427397541645,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541653,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":268008,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2108,"Index":105,"Attempt":0,"Launch Time":1427397541656,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2105,"Index":102,"Attempt":0,"Launch Time":1427397541646,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541656,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":352616,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2109,"Index":106,"Attempt":0,"Launch Time":1427397541656,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2102,"Index":99,"Attempt":0,"Launch Time":1427397541643,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541656,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":303031,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2110,"Index":107,"Attempt":0,"Launch Time":1427397541658,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2106,"Index":103,"Attempt":0,"Launch Time":1427397541650,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541658,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":234805,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2111,"Index":108,"Attempt":0,"Launch Time":1427397541661,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2112,"Index":109,"Attempt":0,"Launch Time":1427397541661,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2108,"Index":105,"Attempt":0,"Launch Time":1427397541656,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541662,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":296499,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2103,"Index":100,"Attempt":0,"Launch Time":1427397541644,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541662,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":275726,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2113,"Index":110,"Attempt":0,"Launch Time":1427397541663,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2100,"Index":97,"Attempt":0,"Launch Time":1427397541641,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541663,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":22,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":326980,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2114,"Index":111,"Attempt":0,"Launch Time":1427397541664,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2109,"Index":106,"Attempt":0,"Launch Time":1427397541656,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541665,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":302224,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2115,"Index":112,"Attempt":0,"Launch Time":1427397541665,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2110,"Index":107,"Attempt":0,"Launch Time":1427397541658,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541665,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":338381,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2116,"Index":113,"Attempt":0,"Launch Time":1427397541666,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2099,"Index":96,"Attempt":0,"Launch Time":1427397541641,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541666,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":19,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":341813,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2117,"Index":114,"Attempt":0,"Launch Time":1427397541668,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2112,"Index":109,"Attempt":0,"Launch Time":1427397541661,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541668,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":317920,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2118,"Index":115,"Attempt":0,"Launch Time":1427397541668,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2111,"Index":108,"Attempt":0,"Launch Time":1427397541661,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541669,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":310154,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2119,"Index":116,"Attempt":0,"Launch Time":1427397541671,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2114,"Index":111,"Attempt":0,"Launch Time":1427397541664,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541671,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":289508,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2120,"Index":117,"Attempt":0,"Launch Time":1427397541672,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2115,"Index":112,"Attempt":0,"Launch Time":1427397541665,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541672,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":285759,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2121,"Index":118,"Attempt":0,"Launch Time":1427397541675,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2117,"Index":114,"Attempt":0,"Launch Time":1427397541668,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541675,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":303502,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2122,"Index":119,"Attempt":0,"Launch Time":1427397541675,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2118,"Index":115,"Attempt":0,"Launch Time":1427397541668,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541676,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":363899,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2123,"Index":120,"Attempt":0,"Launch Time":1427397541677,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2107,"Index":104,"Attempt":0,"Launch Time":1427397541652,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541678,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":351985,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2124,"Index":121,"Attempt":0,"Launch Time":1427397541679,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2120,"Index":117,"Attempt":0,"Launch Time":1427397541672,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541679,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":322497,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2125,"Index":122,"Attempt":0,"Launch Time":1427397541681,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2116,"Index":113,"Attempt":0,"Launch Time":1427397541666,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541681,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1947325,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2119,"Index":116,"Attempt":0,"Launch Time":1427397541671,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541681,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":305331,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2126,"Index":123,"Attempt":0,"Launch Time":1427397541681,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2127,"Index":124,"Attempt":0,"Launch Time":1427397541683,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2122,"Index":119,"Attempt":0,"Launch Time":1427397541675,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541683,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":682299,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2128,"Index":125,"Attempt":0,"Launch Time":1427397541684,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2123,"Index":120,"Attempt":0,"Launch Time":1427397541677,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541684,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":301228,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2129,"Index":126,"Attempt":0,"Launch Time":1427397541685,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2121,"Index":118,"Attempt":0,"Launch Time":1427397541675,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541685,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":314564,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2130,"Index":127,"Attempt":0,"Launch Time":1427397541688,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2126,"Index":123,"Attempt":0,"Launch Time":1427397541681,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541688,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":276277,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2131,"Index":128,"Attempt":0,"Launch Time":1427397541689,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2124,"Index":121,"Attempt":0,"Launch Time":1427397541679,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541690,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":331643,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2132,"Index":129,"Attempt":0,"Launch Time":1427397541690,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2127,"Index":124,"Attempt":0,"Launch Time":1427397541683,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541690,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":280591,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2133,"Index":130,"Attempt":0,"Launch Time":1427397541694,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2134,"Index":131,"Attempt":0,"Launch Time":1427397541694,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2128,"Index":125,"Attempt":0,"Launch Time":1427397541684,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541694,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":287099,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2130,"Index":127,"Attempt":0,"Launch Time":1427397541688,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541694,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":285980,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2135,"Index":132,"Attempt":0,"Launch Time":1427397541696,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2136,"Index":133,"Attempt":0,"Launch Time":1427397541696,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2131,"Index":128,"Attempt":0,"Launch Time":1427397541689,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541696,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":503242,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2132,"Index":129,"Attempt":0,"Launch Time":1427397541690,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541697,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":296642,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2137,"Index":134,"Attempt":0,"Launch Time":1427397541699,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2129,"Index":126,"Attempt":0,"Launch Time":1427397541685,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541699,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":292813,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2138,"Index":135,"Attempt":0,"Launch Time":1427397541711,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2134,"Index":131,"Attempt":0,"Launch Time":1427397541694,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541711,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":333075,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2139,"Index":136,"Attempt":0,"Launch Time":1427397541712,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2133,"Index":130,"Attempt":0,"Launch Time":1427397541694,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541712,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":258059,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2140,"Index":137,"Attempt":0,"Launch Time":1427397541715,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2125,"Index":122,"Attempt":0,"Launch Time":1427397541681,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541715,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":32,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":2002187,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2141,"Index":138,"Attempt":0,"Launch Time":1427397541715,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2096,"Index":93,"Attempt":0,"Launch Time":1427397541636,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541715,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":67,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":323698,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2142,"Index":139,"Attempt":0,"Launch Time":1427397541717,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2137,"Index":134,"Attempt":0,"Launch Time":1427397541699,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541717,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":294033,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2143,"Index":140,"Attempt":0,"Launch Time":1427397541719,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2113,"Index":110,"Attempt":0,"Launch Time":1427397541663,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541719,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":301383,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2144,"Index":141,"Attempt":0,"Launch Time":1427397541722,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2140,"Index":137,"Attempt":0,"Launch Time":1427397541715,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541722,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":364769,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2145,"Index":142,"Attempt":0,"Launch Time":1427397541723,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2142,"Index":139,"Attempt":0,"Launch Time":1427397541717,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541723,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":316648,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2146,"Index":143,"Attempt":0,"Launch Time":1427397541727,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2143,"Index":140,"Attempt":0,"Launch Time":1427397541719,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541727,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9606,"Shuffle Write Time":310701,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2147,"Index":144,"Attempt":0,"Launch Time":1427397541728,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2141,"Index":138,"Attempt":0,"Launch Time":1427397541715,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541728,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":330495,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2148,"Index":145,"Attempt":0,"Launch Time":1427397541731,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2149,"Index":146,"Attempt":0,"Launch Time":1427397541731,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2145,"Index":142,"Attempt":0,"Launch Time":1427397541723,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541731,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":377831,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2144,"Index":141,"Attempt":0,"Launch Time":1427397541722,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541731,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":672517,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2150,"Index":147,"Attempt":0,"Launch Time":1427397541734,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2147,"Index":144,"Attempt":0,"Launch Time":1427397541728,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541735,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":285634,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2151,"Index":148,"Attempt":0,"Launch Time":1427397541735,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2146,"Index":143,"Attempt":0,"Launch Time":1427397541727,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541735,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":277504,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2152,"Index":149,"Attempt":0,"Launch Time":1427397541737,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2149,"Index":146,"Attempt":0,"Launch Time":1427397541731,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541737,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":270722,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2153,"Index":150,"Attempt":0,"Launch Time":1427397541741,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2151,"Index":148,"Attempt":0,"Launch Time":1427397541735,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541741,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":272992,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2154,"Index":151,"Attempt":0,"Launch Time":1427397541741,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2150,"Index":147,"Attempt":0,"Launch Time":1427397541734,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541741,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":257965,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2155,"Index":152,"Attempt":0,"Launch Time":1427397541743,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2152,"Index":149,"Attempt":0,"Launch Time":1427397541737,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541743,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":290612,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2156,"Index":153,"Attempt":0,"Launch Time":1427397541747,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2153,"Index":150,"Attempt":0,"Launch Time":1427397541741,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541747,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":297791,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2157,"Index":154,"Attempt":0,"Launch Time":1427397541748,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2154,"Index":151,"Attempt":0,"Launch Time":1427397541741,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541749,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":304089,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2158,"Index":155,"Attempt":0,"Launch Time":1427397541757,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2157,"Index":154,"Attempt":0,"Launch Time":1427397541748,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541757,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":309058,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2159,"Index":156,"Attempt":0,"Launch Time":1427397541766,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2158,"Index":155,"Attempt":0,"Launch Time":1427397541757,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541766,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":327319,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2160,"Index":157,"Attempt":0,"Launch Time":1427397541767,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2148,"Index":145,"Attempt":0,"Launch Time":1427397541731,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541768,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":36,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":338544,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2161,"Index":158,"Attempt":0,"Launch Time":1427397541768,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2155,"Index":152,"Attempt":0,"Launch Time":1427397541743,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541768,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":25,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":338755,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2162,"Index":159,"Attempt":0,"Launch Time":1427397541768,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2156,"Index":153,"Attempt":0,"Launch Time":1427397541747,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541769,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":363485,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2163,"Index":160,"Attempt":0,"Launch Time":1427397541773,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2159,"Index":156,"Attempt":0,"Launch Time":1427397541766,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541773,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":358033,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2164,"Index":161,"Attempt":0,"Launch Time":1427397541774,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2160,"Index":157,"Attempt":0,"Launch Time":1427397541767,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541774,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":303647,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2165,"Index":162,"Attempt":0,"Launch Time":1427397541774,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2161,"Index":158,"Attempt":0,"Launch Time":1427397541768,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541775,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":298793,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2166,"Index":163,"Attempt":0,"Launch Time":1427397541776,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2162,"Index":159,"Attempt":0,"Launch Time":1427397541768,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541776,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":305405,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2167,"Index":164,"Attempt":0,"Launch Time":1427397541777,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2138,"Index":135,"Attempt":0,"Launch Time":1427397541711,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541777,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":65,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":3571032,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2168,"Index":165,"Attempt":0,"Launch Time":1427397541780,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2163,"Index":160,"Attempt":0,"Launch Time":1427397541773,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541781,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":347409,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2169,"Index":166,"Attempt":0,"Launch Time":1427397541781,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2164,"Index":161,"Attempt":0,"Launch Time":1427397541774,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541781,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":338101,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2170,"Index":167,"Attempt":0,"Launch Time":1427397541784,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2165,"Index":162,"Attempt":0,"Launch Time":1427397541774,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541784,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":334782,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2171,"Index":168,"Attempt":0,"Launch Time":1427397541784,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2167,"Index":164,"Attempt":0,"Launch Time":1427397541777,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541784,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":347463,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2172,"Index":169,"Attempt":0,"Launch Time":1427397541787,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2166,"Index":163,"Attempt":0,"Launch Time":1427397541776,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541787,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":320097,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2173,"Index":170,"Attempt":0,"Launch Time":1427397541787,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2169,"Index":166,"Attempt":0,"Launch Time":1427397541781,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541788,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":275166,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2174,"Index":171,"Attempt":0,"Launch Time":1427397541790,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2168,"Index":165,"Attempt":0,"Launch Time":1427397541780,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541790,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":306228,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2175,"Index":172,"Attempt":0,"Launch Time":1427397541790,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2171,"Index":168,"Attempt":0,"Launch Time":1427397541784,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541790,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":299856,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2176,"Index":173,"Attempt":0,"Launch Time":1427397541794,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2173,"Index":170,"Attempt":0,"Launch Time":1427397541787,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541794,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":263125,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2177,"Index":174,"Attempt":0,"Launch Time":1427397541796,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2170,"Index":167,"Attempt":0,"Launch Time":1427397541784,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541796,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":640643,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2178,"Index":175,"Attempt":0,"Launch Time":1427397541796,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2174,"Index":171,"Attempt":0,"Launch Time":1427397541790,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541796,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":296776,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2179,"Index":176,"Attempt":0,"Launch Time":1427397541797,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2175,"Index":172,"Attempt":0,"Launch Time":1427397541790,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541797,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":293332,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2180,"Index":177,"Attempt":0,"Launch Time":1427397541800,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2176,"Index":173,"Attempt":0,"Launch Time":1427397541794,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541800,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":272886,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2181,"Index":178,"Attempt":0,"Launch Time":1427397541802,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2177,"Index":174,"Attempt":0,"Launch Time":1427397541796,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541802,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":299566,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2182,"Index":179,"Attempt":0,"Launch Time":1427397541803,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2178,"Index":175,"Attempt":0,"Launch Time":1427397541796,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541803,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":313773,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2183,"Index":180,"Attempt":0,"Launch Time":1427397541816,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2181,"Index":178,"Attempt":0,"Launch Time":1427397541802,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541816,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":323029,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2184,"Index":181,"Attempt":0,"Launch Time":1427397541816,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2179,"Index":176,"Attempt":0,"Launch Time":1427397541797,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541817,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":19,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":307869,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2185,"Index":182,"Attempt":0,"Launch Time":1427397541820,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2180,"Index":177,"Attempt":0,"Launch Time":1427397541800,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541820,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":20,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":373044,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2186,"Index":183,"Attempt":0,"Launch Time":1427397541823,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2139,"Index":136,"Attempt":0,"Launch Time":1427397541712,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541823,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":108,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":306093,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2187,"Index":184,"Attempt":0,"Launch Time":1427397541823,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2188,"Index":185,"Attempt":0,"Launch Time":1427397541823,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2189,"Index":186,"Attempt":0,"Launch Time":1427397541823,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2182,"Index":179,"Attempt":0,"Launch Time":1427397541803,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541824,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":19,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":320099,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2172,"Index":169,"Attempt":0,"Launch Time":1427397541787,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541824,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":399774,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2184,"Index":181,"Attempt":0,"Launch Time":1427397541816,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541825,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":299033,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2190,"Index":187,"Attempt":0,"Launch Time":1427397541825,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2183,"Index":180,"Attempt":0,"Launch Time":1427397541816,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541825,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":309184,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2191,"Index":188,"Attempt":0,"Launch Time":1427397541826,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2185,"Index":182,"Attempt":0,"Launch Time":1427397541820,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541826,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":290898,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2192,"Index":189,"Attempt":0,"Launch Time":1427397541830,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2187,"Index":184,"Attempt":0,"Launch Time":1427397541823,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541830,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":359729,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2193,"Index":190,"Attempt":0,"Launch Time":1427397541831,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2186,"Index":183,"Attempt":0,"Launch Time":1427397541823,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541832,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1228549,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2194,"Index":191,"Attempt":0,"Launch Time":1427397541833,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2195,"Index":192,"Attempt":0,"Launch Time":1427397541833,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2191,"Index":188,"Attempt":0,"Launch Time":1427397541826,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541833,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":359581,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2188,"Index":185,"Attempt":0,"Launch Time":1427397541823,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541833,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":390348,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2196,"Index":193,"Attempt":0,"Launch Time":1427397541835,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2135,"Index":132,"Attempt":0,"Launch Time":1427397541696,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541839,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":137,"Result Size":930,"JVM GC Time":13,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":329342,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2197,"Index":194,"Attempt":0,"Launch Time":1427397541840,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2189,"Index":186,"Attempt":0,"Launch Time":1427397541823,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541840,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":314652,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2198,"Index":195,"Attempt":0,"Launch Time":1427397541840,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2136,"Index":133,"Attempt":0,"Launch Time":1427397541696,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541840,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":143,"Result Size":930,"JVM GC Time":13,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":110603586,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2199,"Index":196,"Attempt":0,"Launch Time":1427397541840,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2190,"Index":187,"Attempt":0,"Launch Time":1427397541825,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541840,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":499027,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2200,"Index":197,"Attempt":0,"Launch Time":1427397541841,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2193,"Index":190,"Attempt":0,"Launch Time":1427397541831,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541841,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":308301,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2201,"Index":198,"Attempt":0,"Launch Time":1427397541848,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2199,"Index":196,"Attempt":0,"Launch Time":1427397541840,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541848,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":574202,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2202,"Index":199,"Attempt":0,"Launch Time":1427397541848,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2196,"Index":193,"Attempt":0,"Launch Time":1427397541835,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541848,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":309753,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2203,"Index":200,"Attempt":0,"Launch Time":1427397541850,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2197,"Index":194,"Attempt":0,"Launch Time":1427397541840,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541851,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":284066,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2204,"Index":201,"Attempt":0,"Launch Time":1427397541853,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2195,"Index":192,"Attempt":0,"Launch Time":1427397541833,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541853,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":19,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":304244,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2205,"Index":202,"Attempt":0,"Launch Time":1427397541854,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2201,"Index":198,"Attempt":0,"Launch Time":1427397541848,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541854,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":311432,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2206,"Index":203,"Attempt":0,"Launch Time":1427397541855,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2202,"Index":199,"Attempt":0,"Launch Time":1427397541848,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541855,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":323220,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2207,"Index":204,"Attempt":0,"Launch Time":1427397541859,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2203,"Index":200,"Attempt":0,"Launch Time":1427397541850,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541860,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":397446,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2208,"Index":205,"Attempt":0,"Launch Time":1427397541862,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2205,"Index":202,"Attempt":0,"Launch Time":1427397541854,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541862,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":364576,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2209,"Index":206,"Attempt":0,"Launch Time":1427397541862,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2204,"Index":201,"Attempt":0,"Launch Time":1427397541853,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541862,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":355455,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2210,"Index":207,"Attempt":0,"Launch Time":1427397541862,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2206,"Index":203,"Attempt":0,"Launch Time":1427397541855,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541862,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":313555,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2211,"Index":208,"Attempt":0,"Launch Time":1427397541866,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2207,"Index":204,"Attempt":0,"Launch Time":1427397541859,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541866,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":306816,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2212,"Index":209,"Attempt":0,"Launch Time":1427397541869,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2209,"Index":206,"Attempt":0,"Launch Time":1427397541862,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541869,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":326738,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2213,"Index":210,"Attempt":0,"Launch Time":1427397541869,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2210,"Index":207,"Attempt":0,"Launch Time":1427397541862,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541869,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":324580,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2214,"Index":211,"Attempt":0,"Launch Time":1427397541870,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2208,"Index":205,"Attempt":0,"Launch Time":1427397541862,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541871,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":322221,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2215,"Index":212,"Attempt":0,"Launch Time":1427397541872,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2211,"Index":208,"Attempt":0,"Launch Time":1427397541866,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541872,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":318176,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2216,"Index":213,"Attempt":0,"Launch Time":1427397541878,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2212,"Index":209,"Attempt":0,"Launch Time":1427397541869,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541878,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":302200,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2217,"Index":214,"Attempt":0,"Launch Time":1427397541878,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2214,"Index":211,"Attempt":0,"Launch Time":1427397541870,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541878,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":307556,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2218,"Index":215,"Attempt":0,"Launch Time":1427397541879,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2213,"Index":210,"Attempt":0,"Launch Time":1427397541869,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541879,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":326099,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2219,"Index":216,"Attempt":0,"Launch Time":1427397541879,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2215,"Index":212,"Attempt":0,"Launch Time":1427397541872,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541879,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":306912,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2220,"Index":217,"Attempt":0,"Launch Time":1427397541886,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2217,"Index":214,"Attempt":0,"Launch Time":1427397541878,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541886,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":340950,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2221,"Index":218,"Attempt":0,"Launch Time":1427397541887,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2222,"Index":219,"Attempt":0,"Launch Time":1427397541887,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2218,"Index":215,"Attempt":0,"Launch Time":1427397541879,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541887,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":348881,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2216,"Index":213,"Attempt":0,"Launch Time":1427397541878,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541887,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":328862,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2223,"Index":220,"Attempt":0,"Launch Time":1427397541887,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2219,"Index":216,"Attempt":0,"Launch Time":1427397541879,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541887,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":636809,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2224,"Index":221,"Attempt":0,"Launch Time":1427397541893,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2221,"Index":218,"Attempt":0,"Launch Time":1427397541887,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541893,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":316372,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2225,"Index":222,"Attempt":0,"Launch Time":1427397541894,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2223,"Index":220,"Attempt":0,"Launch Time":1427397541887,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541894,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":315412,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2226,"Index":223,"Attempt":0,"Launch Time":1427397541894,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2220,"Index":217,"Attempt":0,"Launch Time":1427397541886,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541894,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":400033,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2227,"Index":224,"Attempt":0,"Launch Time":1427397541896,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2198,"Index":195,"Attempt":0,"Launch Time":1427397541840,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541896,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":6,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":373470,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2228,"Index":225,"Attempt":0,"Launch Time":1427397541909,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2222,"Index":219,"Attempt":0,"Launch Time":1427397541887,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541909,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":851978,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2229,"Index":226,"Attempt":0,"Launch Time":1427397541909,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2225,"Index":222,"Attempt":0,"Launch Time":1427397541894,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541910,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":345265,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2230,"Index":227,"Attempt":0,"Launch Time":1427397541912,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2224,"Index":221,"Attempt":0,"Launch Time":1427397541893,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541912,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":376210,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2231,"Index":228,"Attempt":0,"Launch Time":1427397541912,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2226,"Index":223,"Attempt":0,"Launch Time":1427397541894,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541913,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":355807,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2232,"Index":229,"Attempt":0,"Launch Time":1427397541917,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2229,"Index":226,"Attempt":0,"Launch Time":1427397541909,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541917,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":340542,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2233,"Index":230,"Attempt":0,"Launch Time":1427397541917,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2228,"Index":225,"Attempt":0,"Launch Time":1427397541909,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541917,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":437916,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2234,"Index":231,"Attempt":0,"Launch Time":1427397541918,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2227,"Index":224,"Attempt":0,"Launch Time":1427397541896,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541918,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1112626,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2235,"Index":232,"Attempt":0,"Launch Time":1427397541919,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2230,"Index":227,"Attempt":0,"Launch Time":1427397541912,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541919,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":321950,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2236,"Index":233,"Attempt":0,"Launch Time":1427397541919,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2231,"Index":228,"Attempt":0,"Launch Time":1427397541912,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541920,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":341016,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2237,"Index":234,"Attempt":0,"Launch Time":1427397541924,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2232,"Index":229,"Attempt":0,"Launch Time":1427397541917,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541924,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":320983,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2238,"Index":235,"Attempt":0,"Launch Time":1427397541926,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2233,"Index":230,"Attempt":0,"Launch Time":1427397541917,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541926,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":317448,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2239,"Index":236,"Attempt":0,"Launch Time":1427397541927,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2235,"Index":232,"Attempt":0,"Launch Time":1427397541919,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541927,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":335610,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2240,"Index":237,"Attempt":0,"Launch Time":1427397541927,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2236,"Index":233,"Attempt":0,"Launch Time":1427397541919,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541927,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":321108,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2241,"Index":238,"Attempt":0,"Launch Time":1427397541929,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2234,"Index":231,"Attempt":0,"Launch Time":1427397541918,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541929,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":356579,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2242,"Index":239,"Attempt":0,"Launch Time":1427397541931,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2237,"Index":234,"Attempt":0,"Launch Time":1427397541924,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541931,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":395576,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2243,"Index":240,"Attempt":0,"Launch Time":1427397541932,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2238,"Index":235,"Attempt":0,"Launch Time":1427397541926,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541933,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":344282,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2244,"Index":241,"Attempt":0,"Launch Time":1427397541941,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2240,"Index":237,"Attempt":0,"Launch Time":1427397541927,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541941,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":411742,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2245,"Index":242,"Attempt":0,"Launch Time":1427397541941,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2242,"Index":239,"Attempt":0,"Launch Time":1427397541931,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541941,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":325035,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2246,"Index":243,"Attempt":0,"Launch Time":1427397541942,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2247,"Index":244,"Attempt":0,"Launch Time":1427397541942,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2192,"Index":189,"Attempt":0,"Launch Time":1427397541830,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541942,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":110,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":323743,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2239,"Index":236,"Attempt":0,"Launch Time":1427397541927,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541942,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":326668,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2248,"Index":245,"Attempt":0,"Launch Time":1427397541942,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2249,"Index":246,"Attempt":0,"Launch Time":1427397541942,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2243,"Index":240,"Attempt":0,"Launch Time":1427397541932,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541942,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":478244,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2241,"Index":238,"Attempt":0,"Launch Time":1427397541929,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541942,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":376472,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2250,"Index":247,"Attempt":0,"Launch Time":1427397541951,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2249,"Index":246,"Attempt":0,"Launch Time":1427397541942,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541951,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":394693,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2251,"Index":248,"Attempt":0,"Launch Time":1427397541952,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2246,"Index":243,"Attempt":0,"Launch Time":1427397541942,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541952,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":469396,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2252,"Index":249,"Attempt":0,"Launch Time":1427397541952,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2244,"Index":241,"Attempt":0,"Launch Time":1427397541941,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541952,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":409163,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2253,"Index":250,"Attempt":0,"Launch Time":1427397541952,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2248,"Index":245,"Attempt":0,"Launch Time":1427397541942,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541952,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":511449,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2254,"Index":251,"Attempt":0,"Launch Time":1427397541954,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2200,"Index":197,"Attempt":0,"Launch Time":1427397541841,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541954,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":47,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":652169,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2245,"Index":242,"Attempt":0,"Launch Time":1427397541941,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541954,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":357144,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2255,"Index":252,"Attempt":0,"Launch Time":1427397541954,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2256,"Index":253,"Attempt":0,"Launch Time":1427397541958,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2250,"Index":247,"Attempt":0,"Launch Time":1427397541951,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541958,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":302222,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2257,"Index":254,"Attempt":0,"Launch Time":1427397541959,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2252,"Index":249,"Attempt":0,"Launch Time":1427397541952,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541959,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9613,"Shuffle Write Time":344191,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2258,"Index":255,"Attempt":0,"Launch Time":1427397541959,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2251,"Index":248,"Attempt":0,"Launch Time":1427397541952,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541960,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":312262,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2259,"Index":256,"Attempt":0,"Launch Time":1427397541962,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2253,"Index":250,"Attempt":0,"Launch Time":1427397541952,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541962,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":458451,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2260,"Index":257,"Attempt":0,"Launch Time":1427397541963,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2254,"Index":251,"Attempt":0,"Launch Time":1427397541954,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541963,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":355736,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2261,"Index":258,"Attempt":0,"Launch Time":1427397541966,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2256,"Index":253,"Attempt":0,"Launch Time":1427397541958,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541966,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":392625,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2262,"Index":259,"Attempt":0,"Launch Time":1427397541966,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2257,"Index":254,"Attempt":0,"Launch Time":1427397541959,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541966,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":333854,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2263,"Index":260,"Attempt":0,"Launch Time":1427397541966,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2258,"Index":255,"Attempt":0,"Launch Time":1427397541959,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541966,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":291005,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2264,"Index":261,"Attempt":0,"Launch Time":1427397541970,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2255,"Index":252,"Attempt":0,"Launch Time":1427397541954,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541970,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":290937,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2265,"Index":262,"Attempt":0,"Launch Time":1427397541970,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2247,"Index":244,"Attempt":0,"Launch Time":1427397541942,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541971,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":16,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":285444,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2266,"Index":263,"Attempt":0,"Launch Time":1427397541972,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2194,"Index":191,"Attempt":0,"Launch Time":1427397541833,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541973,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":139,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":344423,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2267,"Index":264,"Attempt":0,"Launch Time":1427397541977,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2268,"Index":265,"Attempt":0,"Launch Time":1427397541977,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2265,"Index":262,"Attempt":0,"Launch Time":1427397541970,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541977,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":329645,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2264,"Index":261,"Attempt":0,"Launch Time":1427397541970,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541977,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":336854,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2269,"Index":266,"Attempt":0,"Launch Time":1427397541983,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2270,"Index":267,"Attempt":0,"Launch Time":1427397541984,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2267,"Index":264,"Attempt":0,"Launch Time":1427397541977,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541984,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":362817,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2268,"Index":265,"Attempt":0,"Launch Time":1427397541977,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541984,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9613,"Shuffle Write Time":309345,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2271,"Index":268,"Attempt":0,"Launch Time":1427397541985,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2266,"Index":263,"Attempt":0,"Launch Time":1427397541972,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541985,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":302426,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2272,"Index":269,"Attempt":0,"Launch Time":1427397541992,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2269,"Index":266,"Attempt":0,"Launch Time":1427397541983,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541992,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":335486,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2273,"Index":270,"Attempt":0,"Launch Time":1427397541992,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2270,"Index":267,"Attempt":0,"Launch Time":1427397541984,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541992,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":393320,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2274,"Index":271,"Attempt":0,"Launch Time":1427397542004,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2271,"Index":268,"Attempt":0,"Launch Time":1427397541985,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542005,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":419921,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2275,"Index":272,"Attempt":0,"Launch Time":1427397542005,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2273,"Index":270,"Attempt":0,"Launch Time":1427397541992,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542005,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":345421,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2276,"Index":273,"Attempt":0,"Launch Time":1427397542008,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2272,"Index":269,"Attempt":0,"Launch Time":1427397541992,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542008,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":15,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":8147200,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2277,"Index":274,"Attempt":0,"Launch Time":1427397542014,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2275,"Index":272,"Attempt":0,"Launch Time":1427397542005,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542014,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":306303,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2278,"Index":275,"Attempt":0,"Launch Time":1427397542015,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2274,"Index":271,"Attempt":0,"Launch Time":1427397542004,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542015,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":324117,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2279,"Index":276,"Attempt":0,"Launch Time":1427397542016,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2276,"Index":273,"Attempt":0,"Launch Time":1427397542008,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542016,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":330288,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2280,"Index":277,"Attempt":0,"Launch Time":1427397542019,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2260,"Index":257,"Attempt":0,"Launch Time":1427397541963,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542020,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":25,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9613,"Shuffle Write Time":566607,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2281,"Index":278,"Attempt":0,"Launch Time":1427397542021,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2282,"Index":279,"Attempt":0,"Launch Time":1427397542021,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2277,"Index":274,"Attempt":0,"Launch Time":1427397542014,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542021,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":344946,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2278,"Index":275,"Attempt":0,"Launch Time":1427397542015,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542022,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":330207,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2283,"Index":280,"Attempt":0,"Launch Time":1427397542023,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2279,"Index":276,"Attempt":0,"Launch Time":1427397542016,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542023,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":305794,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2284,"Index":281,"Attempt":0,"Launch Time":1427397542028,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2285,"Index":282,"Attempt":0,"Launch Time":1427397542028,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2280,"Index":277,"Attempt":0,"Launch Time":1427397542019,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542029,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":311465,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2281,"Index":278,"Attempt":0,"Launch Time":1427397542021,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542029,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":371639,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2286,"Index":283,"Attempt":0,"Launch Time":1427397542030,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2283,"Index":280,"Attempt":0,"Launch Time":1427397542023,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542031,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":346122,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2287,"Index":284,"Attempt":0,"Launch Time":1427397542035,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2285,"Index":282,"Attempt":0,"Launch Time":1427397542028,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542035,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":338520,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2288,"Index":285,"Attempt":0,"Launch Time":1427397542037,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2284,"Index":281,"Attempt":0,"Launch Time":1427397542028,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542037,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":343835,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2289,"Index":286,"Attempt":0,"Launch Time":1427397542039,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2286,"Index":283,"Attempt":0,"Launch Time":1427397542030,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542039,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":342146,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2290,"Index":287,"Attempt":0,"Launch Time":1427397542043,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2291,"Index":288,"Attempt":0,"Launch Time":1427397542044,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2287,"Index":284,"Attempt":0,"Launch Time":1427397542035,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542044,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":381503,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2259,"Index":256,"Attempt":0,"Launch Time":1427397541962,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542044,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":5,"Executor Run Time":34,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":606222,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2292,"Index":289,"Attempt":0,"Launch Time":1427397542046,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2293,"Index":290,"Attempt":0,"Launch Time":1427397542046,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2289,"Index":286,"Attempt":0,"Launch Time":1427397542039,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542046,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":379140,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2288,"Index":285,"Attempt":0,"Launch Time":1427397542037,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542048,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":530244,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2294,"Index":291,"Attempt":0,"Launch Time":1427397542052,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2291,"Index":288,"Attempt":0,"Launch Time":1427397542044,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542052,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":920799,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2295,"Index":292,"Attempt":0,"Launch Time":1427397542053,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2293,"Index":290,"Attempt":0,"Launch Time":1427397542046,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542053,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":317269,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2296,"Index":293,"Attempt":0,"Launch Time":1427397542054,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2292,"Index":289,"Attempt":0,"Launch Time":1427397542046,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542054,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":307930,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2297,"Index":294,"Attempt":0,"Launch Time":1427397542059,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2290,"Index":287,"Attempt":0,"Launch Time":1427397542043,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542060,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":378068,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2298,"Index":295,"Attempt":0,"Launch Time":1427397542060,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2295,"Index":292,"Attempt":0,"Launch Time":1427397542053,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542060,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":327275,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2299,"Index":296,"Attempt":0,"Launch Time":1427397542060,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2294,"Index":291,"Attempt":0,"Launch Time":1427397542052,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542060,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":359523,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2300,"Index":297,"Attempt":0,"Launch Time":1427397542061,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2296,"Index":293,"Attempt":0,"Launch Time":1427397542054,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542061,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":359912,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2301,"Index":298,"Attempt":0,"Launch Time":1427397542067,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2298,"Index":295,"Attempt":0,"Launch Time":1427397542060,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542067,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":376475,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2302,"Index":299,"Attempt":0,"Launch Time":1427397542068,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2300,"Index":297,"Attempt":0,"Launch Time":1427397542061,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542069,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":323995,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2303,"Index":300,"Attempt":0,"Launch Time":1427397542070,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2299,"Index":296,"Attempt":0,"Launch Time":1427397542060,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542070,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":322139,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2304,"Index":301,"Attempt":0,"Launch Time":1427397542074,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2301,"Index":298,"Attempt":0,"Launch Time":1427397542067,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542074,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":323713,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2305,"Index":302,"Attempt":0,"Launch Time":1427397542077,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2302,"Index":299,"Attempt":0,"Launch Time":1427397542068,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542077,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":396110,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2306,"Index":303,"Attempt":0,"Launch Time":1427397542078,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2303,"Index":300,"Attempt":0,"Launch Time":1427397542070,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542078,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":343527,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2307,"Index":304,"Attempt":0,"Launch Time":1427397542081,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2304,"Index":301,"Attempt":0,"Launch Time":1427397542074,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542081,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":326302,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2308,"Index":305,"Attempt":0,"Launch Time":1427397542085,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2306,"Index":303,"Attempt":0,"Launch Time":1427397542078,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542085,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":296646,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2309,"Index":306,"Attempt":0,"Launch Time":1427397542087,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2305,"Index":302,"Attempt":0,"Launch Time":1427397542077,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542087,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":336300,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2310,"Index":307,"Attempt":0,"Launch Time":1427397542088,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2307,"Index":304,"Attempt":0,"Launch Time":1427397542081,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542088,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":328348,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2311,"Index":308,"Attempt":0,"Launch Time":1427397542089,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2263,"Index":260,"Attempt":0,"Launch Time":1427397541966,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542090,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":41,"Executor Run Time":44,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":335725,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2312,"Index":309,"Attempt":0,"Launch Time":1427397542094,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2262,"Index":259,"Attempt":0,"Launch Time":1427397541966,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542094,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":69,"Executor Run Time":22,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":334446,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2313,"Index":310,"Attempt":0,"Launch Time":1427397542095,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2261,"Index":258,"Attempt":0,"Launch Time":1427397541966,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542095,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":91,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":442406,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2314,"Index":311,"Attempt":0,"Launch Time":1427397542097,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2297,"Index":294,"Attempt":0,"Launch Time":1427397542059,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542097,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":34,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":343556,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2315,"Index":312,"Attempt":0,"Launch Time":1427397542097,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2282,"Index":279,"Attempt":0,"Launch Time":1427397542021,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542097,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":75,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1147999,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2316,"Index":313,"Attempt":0,"Launch Time":1427397542101,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2317,"Index":314,"Attempt":0,"Launch Time":1427397542101,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2311,"Index":308,"Attempt":0,"Launch Time":1427397542089,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542101,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":324538,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2312,"Index":309,"Attempt":0,"Launch Time":1427397542094,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542101,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":282174,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2318,"Index":315,"Attempt":0,"Launch Time":1427397542110,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2315,"Index":312,"Attempt":0,"Launch Time":1427397542097,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542110,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":342022,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2319,"Index":316,"Attempt":0,"Launch Time":1427397542112,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2314,"Index":311,"Attempt":0,"Launch Time":1427397542097,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542112,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":327025,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2320,"Index":317,"Attempt":0,"Launch Time":1427397542114,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2313,"Index":310,"Attempt":0,"Launch Time":1427397542095,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542114,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":403492,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2321,"Index":318,"Attempt":0,"Launch Time":1427397542116,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2318,"Index":315,"Attempt":0,"Launch Time":1427397542110,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542117,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":322110,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2322,"Index":319,"Attempt":0,"Launch Time":1427397542118,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2310,"Index":307,"Attempt":0,"Launch Time":1427397542088,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542118,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":328811,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2323,"Index":320,"Attempt":0,"Launch Time":1427397542119,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2324,"Index":321,"Attempt":0,"Launch Time":1427397542119,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2316,"Index":313,"Attempt":0,"Launch Time":1427397542101,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542119,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":316223,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2319,"Index":316,"Attempt":0,"Launch Time":1427397542112,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542119,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":318881,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2325,"Index":322,"Attempt":0,"Launch Time":1427397542121,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2320,"Index":317,"Attempt":0,"Launch Time":1427397542114,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542121,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":321218,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2326,"Index":323,"Attempt":0,"Launch Time":1427397542123,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2317,"Index":314,"Attempt":0,"Launch Time":1427397542101,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542123,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":20,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":364994,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2327,"Index":324,"Attempt":0,"Launch Time":1427397542124,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2321,"Index":318,"Attempt":0,"Launch Time":1427397542116,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542124,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":408981,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2328,"Index":325,"Attempt":0,"Launch Time":1427397542126,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2323,"Index":320,"Attempt":0,"Launch Time":1427397542119,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542126,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":296441,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2329,"Index":326,"Attempt":0,"Launch Time":1427397542126,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2324,"Index":321,"Attempt":0,"Launch Time":1427397542119,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542127,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":379292,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2330,"Index":327,"Attempt":0,"Launch Time":1427397542132,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2327,"Index":324,"Attempt":0,"Launch Time":1427397542124,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542133,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":323550,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2331,"Index":328,"Attempt":0,"Launch Time":1427397542133,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2326,"Index":323,"Attempt":0,"Launch Time":1427397542123,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542133,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":497812,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2332,"Index":329,"Attempt":0,"Launch Time":1427397542134,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2329,"Index":326,"Attempt":0,"Launch Time":1427397542126,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542134,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":291504,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2333,"Index":330,"Attempt":0,"Launch Time":1427397542135,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2328,"Index":325,"Attempt":0,"Launch Time":1427397542126,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542135,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":769044,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2334,"Index":331,"Attempt":0,"Launch Time":1427397542136,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2325,"Index":322,"Attempt":0,"Launch Time":1427397542121,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542136,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":455181,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2335,"Index":332,"Attempt":0,"Launch Time":1427397542141,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2331,"Index":328,"Attempt":0,"Launch Time":1427397542133,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542141,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":324059,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2336,"Index":333,"Attempt":0,"Launch Time":1427397542142,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2332,"Index":329,"Attempt":0,"Launch Time":1427397542134,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542142,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":468594,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2337,"Index":334,"Attempt":0,"Launch Time":1427397542145,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2330,"Index":327,"Attempt":0,"Launch Time":1427397542132,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542145,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":475012,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2338,"Index":335,"Attempt":0,"Launch Time":1427397542146,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2334,"Index":331,"Attempt":0,"Launch Time":1427397542136,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542147,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":381891,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2339,"Index":336,"Attempt":0,"Launch Time":1427397542150,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2335,"Index":332,"Attempt":0,"Launch Time":1427397542141,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542150,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":411684,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2340,"Index":337,"Attempt":0,"Launch Time":1427397542150,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2336,"Index":333,"Attempt":0,"Launch Time":1427397542142,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542150,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":382420,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2341,"Index":338,"Attempt":0,"Launch Time":1427397542153,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2333,"Index":330,"Attempt":0,"Launch Time":1427397542135,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542153,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":362935,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2342,"Index":339,"Attempt":0,"Launch Time":1427397542154,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2322,"Index":319,"Attempt":0,"Launch Time":1427397542118,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542154,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":34,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":312845,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2343,"Index":340,"Attempt":0,"Launch Time":1427397542157,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2344,"Index":341,"Attempt":0,"Launch Time":1427397542157,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2345,"Index":342,"Attempt":0,"Launch Time":1427397542157,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2338,"Index":335,"Attempt":0,"Launch Time":1427397542146,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542157,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9650,"Shuffle Write Time":332504,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2337,"Index":334,"Attempt":0,"Launch Time":1427397542145,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542157,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9666,"Shuffle Write Time":616768,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2346,"Index":343,"Attempt":0,"Launch Time":1427397542157,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2339,"Index":336,"Attempt":0,"Launch Time":1427397542150,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542157,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9648,"Shuffle Write Time":305089,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2340,"Index":337,"Attempt":0,"Launch Time":1427397542150,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542157,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9651,"Shuffle Write Time":310392,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2347,"Index":344,"Attempt":0,"Launch Time":1427397542160,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2342,"Index":339,"Attempt":0,"Launch Time":1427397542154,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542161,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9647,"Shuffle Write Time":290805,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2348,"Index":345,"Attempt":0,"Launch Time":1427397542164,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2344,"Index":341,"Attempt":0,"Launch Time":1427397542157,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542164,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":356426,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2349,"Index":346,"Attempt":0,"Launch Time":1427397542165,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2345,"Index":342,"Attempt":0,"Launch Time":1427397542157,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542165,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":371800,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2350,"Index":347,"Attempt":0,"Launch Time":1427397542170,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2346,"Index":343,"Attempt":0,"Launch Time":1427397542157,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542170,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":351367,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2351,"Index":348,"Attempt":0,"Launch Time":1427397542170,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2341,"Index":338,"Attempt":0,"Launch Time":1427397542153,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542170,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9651,"Shuffle Write Time":403849,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2352,"Index":349,"Attempt":0,"Launch Time":1427397542173,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2309,"Index":306,"Attempt":0,"Launch Time":1427397542087,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542173,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":22,"Executor Run Time":63,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":374327,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2353,"Index":350,"Attempt":0,"Launch Time":1427397542173,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2354,"Index":351,"Attempt":0,"Launch Time":1427397542174,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2349,"Index":346,"Attempt":0,"Launch Time":1427397542165,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542174,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":371453,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2348,"Index":345,"Attempt":0,"Launch Time":1427397542164,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542174,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":398153,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2355,"Index":352,"Attempt":0,"Launch Time":1427397542175,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2343,"Index":340,"Attempt":0,"Launch Time":1427397542157,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542175,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9649,"Shuffle Write Time":499926,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2356,"Index":353,"Attempt":0,"Launch Time":1427397542182,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2354,"Index":351,"Attempt":0,"Launch Time":1427397542174,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542182,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":350232,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2353,"Index":350,"Attempt":0,"Launch Time":1427397542173,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542182,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":498006,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2357,"Index":354,"Attempt":0,"Launch Time":1427397542182,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2358,"Index":355,"Attempt":0,"Launch Time":1427397542183,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2350,"Index":347,"Attempt":0,"Launch Time":1427397542170,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542183,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":344175,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2359,"Index":356,"Attempt":0,"Launch Time":1427397542186,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2351,"Index":348,"Attempt":0,"Launch Time":1427397542170,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542186,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":370939,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2360,"Index":357,"Attempt":0,"Launch Time":1427397542186,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2355,"Index":352,"Attempt":0,"Launch Time":1427397542175,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542187,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":363006,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2361,"Index":358,"Attempt":0,"Launch Time":1427397542187,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2347,"Index":344,"Attempt":0,"Launch Time":1427397542160,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542188,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":26,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":435519,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2362,"Index":359,"Attempt":0,"Launch Time":1427397542190,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2357,"Index":354,"Attempt":0,"Launch Time":1427397542182,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542190,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":324809,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2363,"Index":360,"Attempt":0,"Launch Time":1427397542205,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2359,"Index":356,"Attempt":0,"Launch Time":1427397542186,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542207,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":18,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":415391,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2364,"Index":361,"Attempt":0,"Launch Time":1427397542208,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2362,"Index":359,"Attempt":0,"Launch Time":1427397542190,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542208,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":11,"Executor Run Time":7,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":334014,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2365,"Index":362,"Attempt":0,"Launch Time":1427397542209,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2361,"Index":358,"Attempt":0,"Launch Time":1427397542187,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542209,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":20,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1346128,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2366,"Index":363,"Attempt":0,"Launch Time":1427397542210,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2367,"Index":364,"Attempt":0,"Launch Time":1427397542210,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2368,"Index":365,"Attempt":0,"Launch Time":1427397542210,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2356,"Index":353,"Attempt":0,"Launch Time":1427397542182,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542210,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":15,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":370058,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2360,"Index":357,"Attempt":0,"Launch Time":1427397542186,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542210,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":346610,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2358,"Index":355,"Attempt":0,"Launch Time":1427397542183,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542210,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":507468,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2369,"Index":366,"Attempt":0,"Launch Time":1427397542225,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2308,"Index":305,"Attempt":0,"Launch Time":1427397542085,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542225,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":139,"Result Size":930,"JVM GC Time":12,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":344795,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2370,"Index":367,"Attempt":0,"Launch Time":1427397542232,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2369,"Index":366,"Attempt":0,"Launch Time":1427397542225,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542232,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":268473,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2371,"Index":368,"Attempt":0,"Launch Time":1427397542242,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2364,"Index":361,"Attempt":0,"Launch Time":1427397542208,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542242,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":352170,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2372,"Index":369,"Attempt":0,"Launch Time":1427397542242,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2366,"Index":363,"Attempt":0,"Launch Time":1427397542210,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542242,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":331418,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2370,"Index":367,"Attempt":0,"Launch Time":1427397542232,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542242,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":310812,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2373,"Index":370,"Attempt":0,"Launch Time":1427397542243,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2374,"Index":371,"Attempt":0,"Launch Time":1427397542243,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2352,"Index":349,"Attempt":0,"Launch Time":1427397542173,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542243,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":342303,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2365,"Index":362,"Attempt":0,"Launch Time":1427397542209,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542243,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":341559,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2375,"Index":372,"Attempt":0,"Launch Time":1427397542243,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2376,"Index":373,"Attempt":0,"Launch Time":1427397542243,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2377,"Index":374,"Attempt":0,"Launch Time":1427397542244,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2363,"Index":360,"Attempt":0,"Launch Time":1427397542205,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542244,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":347017,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2368,"Index":365,"Attempt":0,"Launch Time":1427397542210,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542244,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":7,"Executor Run Time":21,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":345878,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2378,"Index":375,"Attempt":0,"Launch Time":1427397542251,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2373,"Index":370,"Attempt":0,"Launch Time":1427397542243,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542251,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":365629,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2379,"Index":376,"Attempt":0,"Launch Time":1427397542251,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2374,"Index":371,"Attempt":0,"Launch Time":1427397542243,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542252,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":383856,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2380,"Index":377,"Attempt":0,"Launch Time":1427397542253,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2375,"Index":372,"Attempt":0,"Launch Time":1427397542243,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542253,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":883979,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2381,"Index":378,"Attempt":0,"Launch Time":1427397542255,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2377,"Index":374,"Attempt":0,"Launch Time":1427397542244,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542256,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9612,"Shuffle Write Time":352312,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2382,"Index":379,"Attempt":0,"Launch Time":1427397542258,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2371,"Index":368,"Attempt":0,"Launch Time":1427397542242,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542258,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":579560,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2383,"Index":380,"Attempt":0,"Launch Time":1427397542259,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2378,"Index":375,"Attempt":0,"Launch Time":1427397542251,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542259,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":331073,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2384,"Index":381,"Attempt":0,"Launch Time":1427397542260,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2379,"Index":376,"Attempt":0,"Launch Time":1427397542251,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542260,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":319395,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2385,"Index":382,"Attempt":0,"Launch Time":1427397542262,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2381,"Index":378,"Attempt":0,"Launch Time":1427397542255,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542262,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":322385,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2386,"Index":383,"Attempt":0,"Launch Time":1427397542264,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2380,"Index":377,"Attempt":0,"Launch Time":1427397542253,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542264,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":372657,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2387,"Index":384,"Attempt":0,"Launch Time":1427397542266,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2383,"Index":380,"Attempt":0,"Launch Time":1427397542259,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542266,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":321635,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2388,"Index":385,"Attempt":0,"Launch Time":1427397542267,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2384,"Index":381,"Attempt":0,"Launch Time":1427397542260,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542268,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":335935,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2389,"Index":386,"Attempt":0,"Launch Time":1427397542269,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2385,"Index":382,"Attempt":0,"Launch Time":1427397542262,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542270,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9614,"Shuffle Write Time":290965,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2390,"Index":387,"Attempt":0,"Launch Time":1427397542272,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2387,"Index":384,"Attempt":0,"Launch Time":1427397542266,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542273,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":285313,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2391,"Index":388,"Attempt":0,"Launch Time":1427397542274,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2388,"Index":385,"Attempt":0,"Launch Time":1427397542267,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542274,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":277760,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2392,"Index":389,"Attempt":0,"Launch Time":1427397542275,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2386,"Index":383,"Attempt":0,"Launch Time":1427397542264,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542275,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":415511,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2393,"Index":390,"Attempt":0,"Launch Time":1427397542279,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2394,"Index":391,"Attempt":0,"Launch Time":1427397542279,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2390,"Index":387,"Attempt":0,"Launch Time":1427397542272,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542279,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":323096,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2389,"Index":386,"Attempt":0,"Launch Time":1427397542269,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542279,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":322484,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2395,"Index":392,"Attempt":0,"Launch Time":1427397542280,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2391,"Index":388,"Attempt":0,"Launch Time":1427397542274,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542280,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":295716,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2396,"Index":393,"Attempt":0,"Launch Time":1427397542282,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2392,"Index":389,"Attempt":0,"Launch Time":1427397542275,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542282,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":338158,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2397,"Index":394,"Attempt":0,"Launch Time":1427397542286,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2393,"Index":390,"Attempt":0,"Launch Time":1427397542279,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542286,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9615,"Shuffle Write Time":440435,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2398,"Index":395,"Attempt":0,"Launch Time":1427397542287,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2382,"Index":379,"Attempt":0,"Launch Time":1427397542258,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542287,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":28,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":20792857,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2399,"Index":396,"Attempt":0,"Launch Time":1427397542287,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2395,"Index":392,"Attempt":0,"Launch Time":1427397542280,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542288,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":399357,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2400,"Index":397,"Attempt":0,"Launch Time":1427397542289,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2396,"Index":393,"Attempt":0,"Launch Time":1427397542282,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542289,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":327474,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2401,"Index":398,"Attempt":0,"Launch Time":1427397542291,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2394,"Index":391,"Attempt":0,"Launch Time":1427397542279,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542291,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":445602,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2402,"Index":399,"Attempt":0,"Launch Time":1427397542296,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2397,"Index":394,"Attempt":0,"Launch Time":1427397542286,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542296,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":382343,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2403,"Index":400,"Attempt":0,"Launch Time":1427397542299,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2401,"Index":398,"Attempt":0,"Launch Time":1427397542291,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542299,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":379093,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2404,"Index":401,"Attempt":0,"Launch Time":1427397542299,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2400,"Index":397,"Attempt":0,"Launch Time":1427397542289,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542299,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":527832,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2405,"Index":402,"Attempt":0,"Launch Time":1427397542301,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2372,"Index":369,"Attempt":0,"Launch Time":1427397542242,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542301,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":333535,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2406,"Index":403,"Attempt":0,"Launch Time":1427397542303,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2399,"Index":396,"Attempt":0,"Launch Time":1427397542287,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542303,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1037119,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2407,"Index":404,"Attempt":0,"Launch Time":1427397542313,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2404,"Index":401,"Attempt":0,"Launch Time":1427397542299,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542313,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":347748,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2408,"Index":405,"Attempt":0,"Launch Time":1427397542317,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2367,"Index":364,"Attempt":0,"Launch Time":1427397542210,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542317,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":22,"Executor Run Time":84,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":310685,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2409,"Index":406,"Attempt":0,"Launch Time":1427397542319,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2402,"Index":399,"Attempt":0,"Launch Time":1427397542296,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542319,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":22,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":648398,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2410,"Index":407,"Attempt":0,"Launch Time":1427397542320,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2407,"Index":404,"Attempt":0,"Launch Time":1427397542313,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542320,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":346903,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2411,"Index":408,"Attempt":0,"Launch Time":1427397542324,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2408,"Index":405,"Attempt":0,"Launch Time":1427397542317,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542324,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":326599,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2412,"Index":409,"Attempt":0,"Launch Time":1427397542326,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2403,"Index":400,"Attempt":0,"Launch Time":1427397542299,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542326,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":26,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":19950449,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2413,"Index":410,"Attempt":0,"Launch Time":1427397542327,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2409,"Index":406,"Attempt":0,"Launch Time":1427397542319,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542327,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":329296,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2414,"Index":411,"Attempt":0,"Launch Time":1427397542332,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2410,"Index":407,"Attempt":0,"Launch Time":1427397542320,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542332,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":316230,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2415,"Index":412,"Attempt":0,"Launch Time":1427397542333,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2412,"Index":409,"Attempt":0,"Launch Time":1427397542326,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542333,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":347714,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2416,"Index":413,"Attempt":0,"Launch Time":1427397542339,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2417,"Index":414,"Attempt":0,"Launch Time":1427397542340,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2415,"Index":412,"Attempt":0,"Launch Time":1427397542333,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542340,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":298540,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2414,"Index":411,"Attempt":0,"Launch Time":1427397542332,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542340,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":522391,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2418,"Index":415,"Attempt":0,"Launch Time":1427397542340,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2406,"Index":403,"Attempt":0,"Launch Time":1427397542303,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542341,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":36,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":584072,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2419,"Index":416,"Attempt":0,"Launch Time":1427397542345,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2376,"Index":373,"Attempt":0,"Launch Time":1427397542243,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542345,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":101,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":19188514,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2420,"Index":417,"Attempt":0,"Launch Time":1427397542348,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2416,"Index":413,"Attempt":0,"Launch Time":1427397542339,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542348,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":324268,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2421,"Index":418,"Attempt":0,"Launch Time":1427397542349,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2405,"Index":402,"Attempt":0,"Launch Time":1427397542301,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542349,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":42,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":384859,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2422,"Index":419,"Attempt":0,"Launch Time":1427397542350,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2411,"Index":408,"Attempt":0,"Launch Time":1427397542324,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542350,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":574971,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2423,"Index":420,"Attempt":0,"Launch Time":1427397542355,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2419,"Index":416,"Attempt":0,"Launch Time":1427397542345,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542355,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":350254,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2424,"Index":421,"Attempt":0,"Launch Time":1427397542355,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2420,"Index":417,"Attempt":0,"Launch Time":1427397542348,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542355,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":325718,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2425,"Index":422,"Attempt":0,"Launch Time":1427397542358,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2422,"Index":419,"Attempt":0,"Launch Time":1427397542350,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542358,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":338440,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2426,"Index":423,"Attempt":0,"Launch Time":1427397542358,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2421,"Index":418,"Attempt":0,"Launch Time":1427397542349,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542358,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":325187,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2427,"Index":424,"Attempt":0,"Launch Time":1427397542363,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2423,"Index":420,"Attempt":0,"Launch Time":1427397542355,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542363,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":363548,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2428,"Index":425,"Attempt":0,"Launch Time":1427397542368,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2429,"Index":426,"Attempt":0,"Launch Time":1427397542368,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2426,"Index":423,"Attempt":0,"Launch Time":1427397542358,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542369,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":347669,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2424,"Index":421,"Attempt":0,"Launch Time":1427397542355,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542369,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":406148,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2430,"Index":427,"Attempt":0,"Launch Time":1427397542370,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2425,"Index":422,"Attempt":0,"Launch Time":1427397542358,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542371,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":367111,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2431,"Index":428,"Attempt":0,"Launch Time":1427397542371,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2427,"Index":424,"Attempt":0,"Launch Time":1427397542363,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542371,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":317201,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2432,"Index":429,"Attempt":0,"Launch Time":1427397542377,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2429,"Index":426,"Attempt":0,"Launch Time":1427397542368,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542377,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":303552,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2433,"Index":430,"Attempt":0,"Launch Time":1427397542377,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2428,"Index":425,"Attempt":0,"Launch Time":1427397542368,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542377,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":379299,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2434,"Index":431,"Attempt":0,"Launch Time":1427397542380,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2435,"Index":432,"Attempt":0,"Launch Time":1427397542380,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2398,"Index":395,"Attempt":0,"Launch Time":1427397542287,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542380,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":91,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":552343,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2431,"Index":428,"Attempt":0,"Launch Time":1427397542371,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542380,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":487922,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2436,"Index":433,"Attempt":0,"Launch Time":1427397542381,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2430,"Index":427,"Attempt":0,"Launch Time":1427397542370,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542381,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":337343,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2437,"Index":434,"Attempt":0,"Launch Time":1427397542384,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2432,"Index":429,"Attempt":0,"Launch Time":1427397542377,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542384,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":340308,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2438,"Index":435,"Attempt":0,"Launch Time":1427397542384,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2433,"Index":430,"Attempt":0,"Launch Time":1427397542377,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542385,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":341278,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2439,"Index":436,"Attempt":0,"Launch Time":1427397542388,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2434,"Index":431,"Attempt":0,"Launch Time":1427397542380,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542388,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":334178,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2440,"Index":437,"Attempt":0,"Launch Time":1427397542388,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2436,"Index":433,"Attempt":0,"Launch Time":1427397542381,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542388,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":322231,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2441,"Index":438,"Attempt":0,"Launch Time":1427397542392,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2442,"Index":439,"Attempt":0,"Launch Time":1427397542392,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2438,"Index":435,"Attempt":0,"Launch Time":1427397542384,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542392,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":362229,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2435,"Index":432,"Attempt":0,"Launch Time":1427397542380,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542392,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":568601,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2443,"Index":440,"Attempt":0,"Launch Time":1427397542395,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2437,"Index":434,"Attempt":0,"Launch Time":1427397542384,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542395,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":550818,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2444,"Index":441,"Attempt":0,"Launch Time":1427397542397,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2440,"Index":437,"Attempt":0,"Launch Time":1427397542388,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542397,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":337933,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2445,"Index":442,"Attempt":0,"Launch Time":1427397542398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2439,"Index":436,"Attempt":0,"Launch Time":1427397542388,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542398,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":355278,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2446,"Index":443,"Attempt":0,"Launch Time":1427397542400,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2442,"Index":439,"Attempt":0,"Launch Time":1427397542392,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542400,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":322573,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2447,"Index":444,"Attempt":0,"Launch Time":1427397542400,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2441,"Index":438,"Attempt":0,"Launch Time":1427397542392,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542400,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":309104,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2448,"Index":445,"Attempt":0,"Launch Time":1427397542403,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2443,"Index":440,"Attempt":0,"Launch Time":1427397542395,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542403,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":311109,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2449,"Index":446,"Attempt":0,"Launch Time":1427397542405,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2445,"Index":442,"Attempt":0,"Launch Time":1427397542398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542405,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":322948,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2450,"Index":447,"Attempt":0,"Launch Time":1427397542407,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2451,"Index":448,"Attempt":0,"Launch Time":1427397542407,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2447,"Index":444,"Attempt":0,"Launch Time":1427397542400,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542407,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":317602,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2444,"Index":441,"Attempt":0,"Launch Time":1427397542397,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542407,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":334609,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2452,"Index":449,"Attempt":0,"Launch Time":1427397542408,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2446,"Index":443,"Attempt":0,"Launch Time":1427397542400,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542408,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":496726,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2453,"Index":450,"Attempt":0,"Launch Time":1427397542427,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2448,"Index":445,"Attempt":0,"Launch Time":1427397542403,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542427,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9605,"Shuffle Write Time":322010,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2454,"Index":451,"Attempt":0,"Launch Time":1427397542427,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2450,"Index":447,"Attempt":0,"Launch Time":1427397542407,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542427,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":14,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":321660,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2455,"Index":452,"Attempt":0,"Launch Time":1427397542427,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2451,"Index":448,"Attempt":0,"Launch Time":1427397542407,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542427,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":315222,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2456,"Index":453,"Attempt":0,"Launch Time":1427397542427,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2457,"Index":454,"Attempt":0,"Launch Time":1427397542428,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2458,"Index":455,"Attempt":0,"Launch Time":1427397542428,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2413,"Index":410,"Attempt":0,"Launch Time":1427397542327,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542428,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":95,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":716138,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2452,"Index":449,"Attempt":0,"Launch Time":1427397542408,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542428,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":330028,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2449,"Index":446,"Attempt":0,"Launch Time":1427397542405,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542428,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":310292,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2459,"Index":456,"Attempt":0,"Launch Time":1427397542434,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2455,"Index":452,"Attempt":0,"Launch Time":1427397542427,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542434,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":299568,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2460,"Index":457,"Attempt":0,"Launch Time":1427397542434,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2456,"Index":453,"Attempt":0,"Launch Time":1427397542427,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542434,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":320834,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2461,"Index":458,"Attempt":0,"Launch Time":1427397542437,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2457,"Index":454,"Attempt":0,"Launch Time":1427397542428,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542437,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":331381,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2462,"Index":459,"Attempt":0,"Launch Time":1427397542437,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2454,"Index":451,"Attempt":0,"Launch Time":1427397542427,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542437,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":363713,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2463,"Index":460,"Attempt":0,"Launch Time":1427397542437,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2458,"Index":455,"Attempt":0,"Launch Time":1427397542428,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542438,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":527876,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2464,"Index":461,"Attempt":0,"Launch Time":1427397542441,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2418,"Index":415,"Attempt":0,"Launch Time":1427397542340,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542441,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":9,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":439436,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2465,"Index":462,"Attempt":0,"Launch Time":1427397542442,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2459,"Index":456,"Attempt":0,"Launch Time":1427397542434,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542442,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":333792,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2466,"Index":463,"Attempt":0,"Launch Time":1427397542444,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2467,"Index":464,"Attempt":0,"Launch Time":1427397542445,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2463,"Index":460,"Attempt":0,"Launch Time":1427397542437,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542445,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":376533,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2461,"Index":458,"Attempt":0,"Launch Time":1427397542437,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542445,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":351217,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2462,"Index":459,"Attempt":0,"Launch Time":1427397542437,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542445,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":351688,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2468,"Index":465,"Attempt":0,"Launch Time":1427397542445,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2469,"Index":466,"Attempt":0,"Launch Time":1427397542446,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2460,"Index":457,"Attempt":0,"Launch Time":1427397542434,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542446,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":391038,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2470,"Index":467,"Attempt":0,"Launch Time":1427397542450,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2465,"Index":462,"Attempt":0,"Launch Time":1427397542442,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542450,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":331695,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2471,"Index":468,"Attempt":0,"Launch Time":1427397542452,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2467,"Index":464,"Attempt":0,"Launch Time":1427397542445,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542452,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":326646,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2472,"Index":469,"Attempt":0,"Launch Time":1427397542454,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2466,"Index":463,"Attempt":0,"Launch Time":1427397542444,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542454,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9607,"Shuffle Write Time":346686,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2473,"Index":470,"Attempt":0,"Launch Time":1427397542454,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2468,"Index":465,"Attempt":0,"Launch Time":1427397542445,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542454,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":349101,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2474,"Index":471,"Attempt":0,"Launch Time":1427397542456,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2469,"Index":466,"Attempt":0,"Launch Time":1427397542446,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542456,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":549663,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2475,"Index":472,"Attempt":0,"Launch Time":1427397542458,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2470,"Index":467,"Attempt":0,"Launch Time":1427397542450,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542458,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":320713,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2476,"Index":473,"Attempt":0,"Launch Time":1427397542459,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2471,"Index":468,"Attempt":0,"Launch Time":1427397542452,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542459,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":357909,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2477,"Index":474,"Attempt":0,"Launch Time":1427397542460,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2464,"Index":461,"Attempt":0,"Launch Time":1427397542441,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542461,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":496141,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2478,"Index":475,"Attempt":0,"Launch Time":1427397542461,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2473,"Index":470,"Attempt":0,"Launch Time":1427397542454,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542462,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":410225,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2479,"Index":476,"Attempt":0,"Launch Time":1427397542463,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2472,"Index":469,"Attempt":0,"Launch Time":1427397542454,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542463,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":391733,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2480,"Index":477,"Attempt":0,"Launch Time":1427397542466,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2477,"Index":474,"Attempt":0,"Launch Time":1427397542460,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542467,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":275697,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2481,"Index":478,"Attempt":0,"Launch Time":1427397542468,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2453,"Index":450,"Attempt":0,"Launch Time":1427397542427,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542468,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":36,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":316955,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2482,"Index":479,"Attempt":0,"Launch Time":1427397542473,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2417,"Index":414,"Attempt":0,"Launch Time":1427397542340,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542473,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":46,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":287857,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2483,"Index":480,"Attempt":0,"Launch Time":1427397542475,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2481,"Index":478,"Attempt":0,"Launch Time":1427397542468,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542476,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":410467,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2484,"Index":481,"Attempt":0,"Launch Time":1427397542481,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2482,"Index":479,"Attempt":0,"Launch Time":1427397542473,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542481,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":359072,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2485,"Index":482,"Attempt":0,"Launch Time":1427397542484,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2483,"Index":480,"Attempt":0,"Launch Time":1427397542475,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542484,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":549921,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2486,"Index":483,"Attempt":0,"Launch Time":1427397542484,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2480,"Index":477,"Attempt":0,"Launch Time":1427397542466,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542484,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":394959,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2487,"Index":484,"Attempt":0,"Launch Time":1427397542490,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2484,"Index":481,"Attempt":0,"Launch Time":1427397542481,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542490,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":420379,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2488,"Index":485,"Attempt":0,"Launch Time":1427397542491,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2485,"Index":482,"Attempt":0,"Launch Time":1427397542484,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542491,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":310801,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2489,"Index":486,"Attempt":0,"Launch Time":1427397542495,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2486,"Index":483,"Attempt":0,"Launch Time":1427397542484,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542495,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":391466,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2490,"Index":487,"Attempt":0,"Launch Time":1427397542497,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2491,"Index":488,"Attempt":0,"Launch Time":1427397542497,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2487,"Index":484,"Attempt":0,"Launch Time":1427397542490,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542497,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":323457,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2479,"Index":476,"Attempt":0,"Launch Time":1427397542463,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542498,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":434325,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2492,"Index":489,"Attempt":0,"Launch Time":1427397542499,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2488,"Index":485,"Attempt":0,"Launch Time":1427397542491,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542499,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":359385,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2493,"Index":490,"Attempt":0,"Launch Time":1427397542502,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2494,"Index":491,"Attempt":0,"Launch Time":1427397542502,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2489,"Index":486,"Attempt":0,"Launch Time":1427397542495,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542502,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":340097,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2474,"Index":471,"Attempt":0,"Launch Time":1427397542456,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542502,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":45,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9613,"Shuffle Write Time":348155,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2495,"Index":492,"Attempt":0,"Launch Time":1427397542507,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2496,"Index":493,"Attempt":0,"Launch Time":1427397542507,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2492,"Index":489,"Attempt":0,"Launch Time":1427397542499,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542507,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":304047,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2491,"Index":488,"Attempt":0,"Launch Time":1427397542497,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542507,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":380941,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2497,"Index":494,"Attempt":0,"Launch Time":1427397542516,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2494,"Index":491,"Attempt":0,"Launch Time":1427397542502,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542516,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":337287,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2498,"Index":495,"Attempt":0,"Launch Time":1427397542523,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2476,"Index":473,"Attempt":0,"Launch Time":1427397542459,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542523,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":63,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":700197,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2499,"Index":496,"Attempt":0,"Launch Time":1427397542523,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2496,"Index":493,"Attempt":0,"Launch Time":1427397542507,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542523,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":8,"Executor Run Time":7,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":314413,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2500,"Index":497,"Attempt":0,"Launch Time":1427397542525,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2497,"Index":494,"Attempt":0,"Launch Time":1427397542516,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542525,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":292971,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2501,"Index":498,"Attempt":0,"Launch Time":1427397542529,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2498,"Index":495,"Attempt":0,"Launch Time":1427397542523,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542530,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":328189,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2502,"Index":499,"Attempt":0,"Launch Time":1427397542530,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2499,"Index":496,"Attempt":0,"Launch Time":1427397542523,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542530,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":331357,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2503,"Index":500,"Attempt":0,"Launch Time":1427397542532,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2500,"Index":497,"Attempt":0,"Launch Time":1427397542525,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542532,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":273329,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2504,"Index":501,"Attempt":0,"Launch Time":1427397542536,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2493,"Index":490,"Attempt":0,"Launch Time":1427397542502,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542536,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":33,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":624484,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2505,"Index":502,"Attempt":0,"Launch Time":1427397542538,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2502,"Index":499,"Attempt":0,"Launch Time":1427397542530,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542538,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":348431,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2506,"Index":503,"Attempt":0,"Launch Time":1427397542541,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2490,"Index":487,"Attempt":0,"Launch Time":1427397542497,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542541,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":42,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":510094,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2495,"Index":492,"Attempt":0,"Launch Time":1427397542507,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542541,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":368092,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2507,"Index":504,"Attempt":0,"Launch Time":1427397542541,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2508,"Index":505,"Attempt":0,"Launch Time":1427397542545,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2504,"Index":501,"Attempt":0,"Launch Time":1427397542536,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542545,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":415621,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2509,"Index":506,"Attempt":0,"Launch Time":1427397542546,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2478,"Index":475,"Attempt":0,"Launch Time":1427397542461,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542546,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":83,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":9590263,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2510,"Index":507,"Attempt":0,"Launch Time":1427397542546,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2505,"Index":502,"Attempt":0,"Launch Time":1427397542538,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542547,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":346596,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2511,"Index":508,"Attempt":0,"Launch Time":1427397542548,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2507,"Index":504,"Attempt":0,"Launch Time":1427397542541,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542548,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":326758,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2512,"Index":509,"Attempt":0,"Launch Time":1427397542549,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2506,"Index":503,"Attempt":0,"Launch Time":1427397542541,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542549,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":309781,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2513,"Index":510,"Attempt":0,"Launch Time":1427397542553,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2514,"Index":511,"Attempt":0,"Launch Time":1427397542553,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2510,"Index":507,"Attempt":0,"Launch Time":1427397542546,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542553,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":326938,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2501,"Index":498,"Attempt":0,"Launch Time":1427397542529,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542553,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":23,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":421173,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2515,"Index":512,"Attempt":0,"Launch Time":1427397542555,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2503,"Index":500,"Attempt":0,"Launch Time":1427397542532,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542555,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":22,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":336321,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2516,"Index":513,"Attempt":0,"Launch Time":1427397542559,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2508,"Index":505,"Attempt":0,"Launch Time":1427397542545,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542559,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":377808,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2517,"Index":514,"Attempt":0,"Launch Time":1427397542561,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2515,"Index":512,"Attempt":0,"Launch Time":1427397542555,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542561,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":322419,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2518,"Index":515,"Attempt":0,"Launch Time":1427397542563,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2511,"Index":508,"Attempt":0,"Launch Time":1427397542548,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542563,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":374111,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2519,"Index":516,"Attempt":0,"Launch Time":1427397542564,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2513,"Index":510,"Attempt":0,"Launch Time":1427397542553,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542564,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":416383,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2520,"Index":517,"Attempt":0,"Launch Time":1427397542568,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2517,"Index":514,"Attempt":0,"Launch Time":1427397542561,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542568,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":326149,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2521,"Index":518,"Attempt":0,"Launch Time":1427397542570,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2518,"Index":515,"Attempt":0,"Launch Time":1427397542563,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542570,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":338201,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2522,"Index":519,"Attempt":0,"Launch Time":1427397542572,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2516,"Index":513,"Attempt":0,"Launch Time":1427397542559,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542572,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":319540,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2523,"Index":520,"Attempt":0,"Launch Time":1427397542574,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2519,"Index":516,"Attempt":0,"Launch Time":1427397542564,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542575,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":311846,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2524,"Index":521,"Attempt":0,"Launch Time":1427397542577,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2520,"Index":517,"Attempt":0,"Launch Time":1427397542568,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542577,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":371122,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2525,"Index":522,"Attempt":0,"Launch Time":1427397542579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2521,"Index":518,"Attempt":0,"Launch Time":1427397542570,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542579,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":452035,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2526,"Index":523,"Attempt":0,"Launch Time":1427397542582,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2522,"Index":519,"Attempt":0,"Launch Time":1427397542572,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542582,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":453075,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2527,"Index":524,"Attempt":0,"Launch Time":1427397542583,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2512,"Index":509,"Attempt":0,"Launch Time":1427397542549,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542583,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":33,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":430800,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2528,"Index":525,"Attempt":0,"Launch Time":1427397542583,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2523,"Index":520,"Attempt":0,"Launch Time":1427397542574,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542584,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":373774,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2529,"Index":526,"Attempt":0,"Launch Time":1427397542586,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2524,"Index":521,"Attempt":0,"Launch Time":1427397542577,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542586,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":383801,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2530,"Index":527,"Attempt":0,"Launch Time":1427397542589,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2526,"Index":523,"Attempt":0,"Launch Time":1427397542582,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542589,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":354403,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2531,"Index":528,"Attempt":0,"Launch Time":1427397542591,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542598,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2509,"Index":506,"Attempt":0,"Launch Time":1427397542546,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542591,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":44,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":315518,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2532,"Index":529,"Attempt":0,"Launch Time":1427397542593,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542602,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2475,"Index":472,"Attempt":0,"Launch Time":1427397542458,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542593,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":135,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":315019,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2533,"Index":530,"Attempt":0,"Launch Time":1427397542598,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542610,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2531,"Index":528,"Attempt":0,"Launch Time":1427397542591,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542598,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":342602,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2534,"Index":531,"Attempt":0,"Launch Time":1427397542599,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542610,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2527,"Index":524,"Attempt":0,"Launch Time":1427397542583,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542599,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":435053,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2535,"Index":532,"Attempt":0,"Launch Time":1427397542602,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542610,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2532,"Index":529,"Attempt":0,"Launch Time":1427397542593,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542602,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":339748,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2536,"Index":533,"Attempt":0,"Launch Time":1427397542609,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2514,"Index":511,"Attempt":0,"Launch Time":1427397542553,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542610,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":16,"Executor Run Time":34,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":427739,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2537,"Index":534,"Attempt":0,"Launch Time":1427397542610,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2538,"Index":535,"Attempt":0,"Launch Time":1427397542610,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2535,"Index":532,"Attempt":0,"Launch Time":1427397542602,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542610,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":352160,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2534,"Index":531,"Attempt":0,"Launch Time":1427397542599,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542610,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":556996,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2533,"Index":530,"Attempt":0,"Launch Time":1427397542598,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542610,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":416658,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2539,"Index":536,"Attempt":0,"Launch Time":1427397542613,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2540,"Index":537,"Attempt":0,"Launch Time":1427397542613,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2525,"Index":522,"Attempt":0,"Launch Time":1427397542579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542614,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":345078,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2541,"Index":538,"Attempt":0,"Launch Time":1427397542620,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2540,"Index":537,"Attempt":0,"Launch Time":1427397542613,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542620,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":292106,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2542,"Index":539,"Attempt":0,"Launch Time":1427397542631,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2538,"Index":535,"Attempt":0,"Launch Time":1427397542610,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542632,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":14,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":307471,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2543,"Index":540,"Attempt":0,"Launch Time":1427397542632,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2537,"Index":534,"Attempt":0,"Launch Time":1427397542610,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542632,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":15,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":316493,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2544,"Index":541,"Attempt":0,"Launch Time":1427397542635,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2530,"Index":527,"Attempt":0,"Launch Time":1427397542589,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542635,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":32,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":334280,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2545,"Index":542,"Attempt":0,"Launch Time":1427397542636,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2541,"Index":538,"Attempt":0,"Launch Time":1427397542620,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542636,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":296391,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2546,"Index":543,"Attempt":0,"Launch Time":1427397542636,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2539,"Index":536,"Attempt":0,"Launch Time":1427397542613,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542636,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":19,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":343830,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2547,"Index":544,"Attempt":0,"Launch Time":1427397542640,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2543,"Index":540,"Attempt":0,"Launch Time":1427397542632,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542640,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":401032,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2548,"Index":545,"Attempt":0,"Launch Time":1427397542641,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2542,"Index":539,"Attempt":0,"Launch Time":1427397542631,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542642,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":2137446,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2549,"Index":546,"Attempt":0,"Launch Time":1427397542643,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2544,"Index":541,"Attempt":0,"Launch Time":1427397542635,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542643,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":341458,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2550,"Index":547,"Attempt":0,"Launch Time":1427397542643,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2546,"Index":543,"Attempt":0,"Launch Time":1427397542636,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542643,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":320407,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2551,"Index":548,"Attempt":0,"Launch Time":1427397542647,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2547,"Index":544,"Attempt":0,"Launch Time":1427397542640,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542647,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":372874,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2552,"Index":549,"Attempt":0,"Launch Time":1427397542650,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2553,"Index":550,"Attempt":0,"Launch Time":1427397542650,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2548,"Index":545,"Attempt":0,"Launch Time":1427397542641,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542650,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":672859,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2545,"Index":542,"Attempt":0,"Launch Time":1427397542636,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542650,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":807166,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2554,"Index":551,"Attempt":0,"Launch Time":1427397542651,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2555,"Index":552,"Attempt":0,"Launch Time":1427397542651,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2549,"Index":546,"Attempt":0,"Launch Time":1427397542643,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542651,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":351979,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2550,"Index":547,"Attempt":0,"Launch Time":1427397542643,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542651,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":348910,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2556,"Index":553,"Attempt":0,"Launch Time":1427397542654,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2551,"Index":548,"Attempt":0,"Launch Time":1427397542647,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542655,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":273391,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2557,"Index":554,"Attempt":0,"Launch Time":1427397542657,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2552,"Index":549,"Attempt":0,"Launch Time":1427397542650,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542657,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":277446,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2558,"Index":555,"Attempt":0,"Launch Time":1427397542658,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2555,"Index":552,"Attempt":0,"Launch Time":1427397542651,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542658,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":332762,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2559,"Index":556,"Attempt":0,"Launch Time":1427397542659,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2553,"Index":550,"Attempt":0,"Launch Time":1427397542650,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542659,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":330051,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2560,"Index":557,"Attempt":0,"Launch Time":1427397542659,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2554,"Index":551,"Attempt":0,"Launch Time":1427397542651,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542660,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":353880,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2561,"Index":558,"Attempt":0,"Launch Time":1427397542663,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2557,"Index":554,"Attempt":0,"Launch Time":1427397542657,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542663,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":311354,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2562,"Index":559,"Attempt":0,"Launch Time":1427397542667,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2563,"Index":560,"Attempt":0,"Launch Time":1427397542667,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2559,"Index":556,"Attempt":0,"Launch Time":1427397542659,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542667,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":303541,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2560,"Index":557,"Attempt":0,"Launch Time":1427397542659,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542667,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":433393,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2564,"Index":561,"Attempt":0,"Launch Time":1427397542668,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2556,"Index":553,"Attempt":0,"Launch Time":1427397542654,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542668,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":363209,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2565,"Index":562,"Attempt":0,"Launch Time":1427397542670,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2561,"Index":558,"Attempt":0,"Launch Time":1427397542663,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542670,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":317449,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2566,"Index":563,"Attempt":0,"Launch Time":1427397542675,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2562,"Index":559,"Attempt":0,"Launch Time":1427397542667,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542675,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":480584,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2567,"Index":564,"Attempt":0,"Launch Time":1427397542677,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2564,"Index":561,"Attempt":0,"Launch Time":1427397542668,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542677,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":481129,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2568,"Index":565,"Attempt":0,"Launch Time":1427397542677,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2558,"Index":555,"Attempt":0,"Launch Time":1427397542658,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542678,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":402462,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2569,"Index":566,"Attempt":0,"Launch Time":1427397542678,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2565,"Index":562,"Attempt":0,"Launch Time":1427397542670,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542678,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":390115,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2570,"Index":567,"Attempt":0,"Launch Time":1427397542680,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2563,"Index":560,"Attempt":0,"Launch Time":1427397542667,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542680,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":700911,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2571,"Index":568,"Attempt":0,"Launch Time":1427397542684,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2566,"Index":563,"Attempt":0,"Launch Time":1427397542675,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542684,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":402443,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2572,"Index":569,"Attempt":0,"Launch Time":1427397542686,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2567,"Index":564,"Attempt":0,"Launch Time":1427397542677,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542686,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":375877,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2573,"Index":570,"Attempt":0,"Launch Time":1427397542686,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2568,"Index":565,"Attempt":0,"Launch Time":1427397542677,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542686,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":442654,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2574,"Index":571,"Attempt":0,"Launch Time":1427397542687,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2569,"Index":566,"Attempt":0,"Launch Time":1427397542678,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542688,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":353228,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2575,"Index":572,"Attempt":0,"Launch Time":1427397542692,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2571,"Index":568,"Attempt":0,"Launch Time":1427397542684,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542692,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":581749,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2576,"Index":573,"Attempt":0,"Launch Time":1427397542694,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2577,"Index":574,"Attempt":0,"Launch Time":1427397542695,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2570,"Index":567,"Attempt":0,"Launch Time":1427397542680,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542695,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":360718,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2574,"Index":571,"Attempt":0,"Launch Time":1427397542687,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542695,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":386982,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2573,"Index":570,"Attempt":0,"Launch Time":1427397542686,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542695,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":454222,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2578,"Index":575,"Attempt":0,"Launch Time":1427397542695,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2579,"Index":576,"Attempt":0,"Launch Time":1427397542695,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2580,"Index":577,"Attempt":0,"Launch Time":1427397542696,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2572,"Index":569,"Attempt":0,"Launch Time":1427397542686,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542696,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":399381,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2536,"Index":533,"Attempt":0,"Launch Time":1427397542609,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542696,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":49,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":331262,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2581,"Index":578,"Attempt":0,"Launch Time":1427397542700,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2575,"Index":572,"Attempt":0,"Launch Time":1427397542692,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542700,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9614,"Shuffle Write Time":388251,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2582,"Index":579,"Attempt":0,"Launch Time":1427397542704,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2583,"Index":580,"Attempt":0,"Launch Time":1427397542704,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2578,"Index":575,"Attempt":0,"Launch Time":1427397542695,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542704,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":365425,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2580,"Index":577,"Attempt":0,"Launch Time":1427397542696,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542705,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":373166,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2584,"Index":581,"Attempt":0,"Launch Time":1427397542705,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2579,"Index":576,"Attempt":0,"Launch Time":1427397542695,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542705,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":410149,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2585,"Index":582,"Attempt":0,"Launch Time":1427397542707,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2581,"Index":578,"Attempt":0,"Launch Time":1427397542700,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542707,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":290919,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2586,"Index":583,"Attempt":0,"Launch Time":1427397542708,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2577,"Index":574,"Attempt":0,"Launch Time":1427397542695,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542709,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":310991,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2587,"Index":584,"Attempt":0,"Launch Time":1427397542711,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2588,"Index":585,"Attempt":0,"Launch Time":1427397542711,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2582,"Index":579,"Attempt":0,"Launch Time":1427397542704,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542711,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":329215,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2583,"Index":580,"Attempt":0,"Launch Time":1427397542704,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542711,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9611,"Shuffle Write Time":312850,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2589,"Index":586,"Attempt":0,"Launch Time":1427397542712,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2584,"Index":581,"Attempt":0,"Launch Time":1427397542705,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542719,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":329984,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2590,"Index":587,"Attempt":0,"Launch Time":1427397542722,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2528,"Index":525,"Attempt":0,"Launch Time":1427397542583,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542722,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":138,"Result Size":930,"JVM GC Time":11,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":40517475,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2591,"Index":588,"Attempt":0,"Launch Time":1427397542729,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2529,"Index":526,"Attempt":0,"Launch Time":1427397542586,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542729,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":70,"Executor Run Time":49,"Result Size":930,"JVM GC Time":11,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":357247,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2592,"Index":589,"Attempt":0,"Launch Time":1427397542732,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2585,"Index":582,"Attempt":0,"Launch Time":1427397542707,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542732,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":20,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":337896,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2593,"Index":590,"Attempt":0,"Launch Time":1427397542734,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2590,"Index":587,"Attempt":0,"Launch Time":1427397542722,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542734,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":377828,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2594,"Index":591,"Attempt":0,"Launch Time":1427397542736,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2576,"Index":573,"Attempt":0,"Launch Time":1427397542694,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542736,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":25,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":1063619,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2595,"Index":592,"Attempt":0,"Launch Time":1427397542736,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2591,"Index":588,"Attempt":0,"Launch Time":1427397542729,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542736,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9608,"Shuffle Write Time":340638,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2596,"Index":593,"Attempt":0,"Launch Time":1427397542744,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2593,"Index":590,"Attempt":0,"Launch Time":1427397542734,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542745,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":751198,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2597,"Index":594,"Attempt":0,"Launch Time":1427397542745,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2595,"Index":592,"Attempt":0,"Launch Time":1427397542736,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542745,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":698816,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2598,"Index":595,"Attempt":0,"Launch Time":1427397542746,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2594,"Index":591,"Attempt":0,"Launch Time":1427397542736,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542746,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":377558,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2599,"Index":596,"Attempt":0,"Launch Time":1427397542748,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2592,"Index":589,"Attempt":0,"Launch Time":1427397542732,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542748,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":600526,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2600,"Index":597,"Attempt":0,"Launch Time":1427397542752,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2597,"Index":594,"Attempt":0,"Launch Time":1427397542745,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542752,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":314233,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2601,"Index":598,"Attempt":0,"Launch Time":1427397542753,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2598,"Index":595,"Attempt":0,"Launch Time":1427397542746,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542753,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":364206,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2602,"Index":599,"Attempt":0,"Launch Time":1427397542755,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2596,"Index":593,"Attempt":0,"Launch Time":1427397542744,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542755,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":354187,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2603,"Index":600,"Attempt":0,"Launch Time":1427397542759,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2600,"Index":597,"Attempt":0,"Launch Time":1427397542752,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542759,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":357617,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2604,"Index":601,"Attempt":0,"Launch Time":1427397542761,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2599,"Index":596,"Attempt":0,"Launch Time":1427397542748,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542761,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":331695,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2605,"Index":602,"Attempt":0,"Launch Time":1427397542762,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2601,"Index":598,"Attempt":0,"Launch Time":1427397542753,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542762,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":378242,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2606,"Index":603,"Attempt":0,"Launch Time":1427397542763,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2602,"Index":599,"Attempt":0,"Launch Time":1427397542755,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542763,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":316777,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2607,"Index":604,"Attempt":0,"Launch Time":1427397542768,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2603,"Index":600,"Attempt":0,"Launch Time":1427397542759,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542768,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":356251,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2608,"Index":605,"Attempt":0,"Launch Time":1427397542769,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2604,"Index":601,"Attempt":0,"Launch Time":1427397542761,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542769,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":353787,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2609,"Index":606,"Attempt":0,"Launch Time":1427397542771,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2606,"Index":603,"Attempt":0,"Launch Time":1427397542763,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542771,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":357229,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2610,"Index":607,"Attempt":0,"Launch Time":1427397542773,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2605,"Index":602,"Attempt":0,"Launch Time":1427397542762,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542773,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":655953,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2611,"Index":608,"Attempt":0,"Launch Time":1427397542776,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2607,"Index":604,"Attempt":0,"Launch Time":1427397542768,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542777,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":409292,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2612,"Index":609,"Attempt":0,"Launch Time":1427397542778,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2608,"Index":605,"Attempt":0,"Launch Time":1427397542769,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542778,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":392973,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2613,"Index":610,"Attempt":0,"Launch Time":1427397542779,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2609,"Index":606,"Attempt":0,"Launch Time":1427397542771,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542779,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":522248,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2614,"Index":611,"Attempt":0,"Launch Time":1427397542781,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2610,"Index":607,"Attempt":0,"Launch Time":1427397542773,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542782,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":434000,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2615,"Index":612,"Attempt":0,"Launch Time":1427397542784,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2611,"Index":608,"Attempt":0,"Launch Time":1427397542776,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542785,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":385253,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2616,"Index":613,"Attempt":0,"Launch Time":1427397542787,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2612,"Index":609,"Attempt":0,"Launch Time":1427397542778,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542787,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":523803,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2617,"Index":614,"Attempt":0,"Launch Time":1427397542789,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2618,"Index":615,"Attempt":0,"Launch Time":1427397542789,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2613,"Index":610,"Attempt":0,"Launch Time":1427397542779,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542789,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":526643,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2614,"Index":611,"Attempt":0,"Launch Time":1427397542781,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542789,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":339913,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2619,"Index":616,"Attempt":0,"Launch Time":1427397542791,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2615,"Index":612,"Attempt":0,"Launch Time":1427397542784,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542791,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":284400,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2620,"Index":617,"Attempt":0,"Launch Time":1427397542794,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2616,"Index":613,"Attempt":0,"Launch Time":1427397542787,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542794,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":309820,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2621,"Index":618,"Attempt":0,"Launch Time":1427397542798,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2618,"Index":615,"Attempt":0,"Launch Time":1427397542789,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542798,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":412305,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2622,"Index":619,"Attempt":0,"Launch Time":1427397542799,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2619,"Index":616,"Attempt":0,"Launch Time":1427397542791,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542799,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":371645,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2623,"Index":620,"Attempt":0,"Launch Time":1427397542800,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2617,"Index":614,"Attempt":0,"Launch Time":1427397542789,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542800,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":363045,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2624,"Index":621,"Attempt":0,"Launch Time":1427397542801,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2620,"Index":617,"Attempt":0,"Launch Time":1427397542794,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542801,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":346097,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2625,"Index":622,"Attempt":0,"Launch Time":1427397542803,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2589,"Index":586,"Attempt":0,"Launch Time":1427397542712,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542803,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":79,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1202931,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2626,"Index":623,"Attempt":0,"Launch Time":1427397542806,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2621,"Index":618,"Attempt":0,"Launch Time":1427397542798,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542806,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":353449,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2627,"Index":624,"Attempt":0,"Launch Time":1427397542806,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2622,"Index":619,"Attempt":0,"Launch Time":1427397542799,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542806,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":364614,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2628,"Index":625,"Attempt":0,"Launch Time":1427397542825,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2623,"Index":620,"Attempt":0,"Launch Time":1427397542800,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542825,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":370648,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2629,"Index":626,"Attempt":0,"Launch Time":1427397542826,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2587,"Index":584,"Attempt":0,"Launch Time":1427397542711,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542826,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":111,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":579883,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2630,"Index":627,"Attempt":0,"Launch Time":1427397542828,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2631,"Index":628,"Attempt":0,"Launch Time":1427397542828,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2588,"Index":585,"Attempt":0,"Launch Time":1427397542711,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542828,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":109,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":344966,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2627,"Index":624,"Attempt":0,"Launch Time":1427397542806,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542828,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":327972,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2586,"Index":583,"Attempt":0,"Launch Time":1427397542708,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542828,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":109,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":484121,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2632,"Index":629,"Attempt":0,"Launch Time":1427397542835,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2633,"Index":630,"Attempt":0,"Launch Time":1427397542841,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2626,"Index":623,"Attempt":0,"Launch Time":1427397542806,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542841,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":329889,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2634,"Index":631,"Attempt":0,"Launch Time":1427397542841,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2625,"Index":622,"Attempt":0,"Launch Time":1427397542803,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542841,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":311118,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2635,"Index":632,"Attempt":0,"Launch Time":1427397542842,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2624,"Index":621,"Attempt":0,"Launch Time":1427397542801,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542842,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1116695,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2636,"Index":633,"Attempt":0,"Launch Time":1427397542842,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2629,"Index":626,"Attempt":0,"Launch Time":1427397542826,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542842,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":305168,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2630,"Index":627,"Attempt":0,"Launch Time":1427397542828,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542843,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":331391,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2637,"Index":634,"Attempt":0,"Launch Time":1427397542844,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2638,"Index":635,"Attempt":0,"Launch Time":1427397542850,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2628,"Index":625,"Attempt":0,"Launch Time":1427397542825,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542850,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":391646,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2639,"Index":636,"Attempt":0,"Launch Time":1427397542851,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2633,"Index":630,"Attempt":0,"Launch Time":1427397542841,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542851,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":333478,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2640,"Index":637,"Attempt":0,"Launch Time":1427397542851,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2636,"Index":633,"Attempt":0,"Launch Time":1427397542842,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542853,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":338975,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2641,"Index":638,"Attempt":0,"Launch Time":1427397542854,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2642,"Index":639,"Attempt":0,"Launch Time":1427397542860,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2643,"Index":640,"Attempt":0,"Launch Time":1427397542860,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2639,"Index":636,"Attempt":0,"Launch Time":1427397542851,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542861,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":311618,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2637,"Index":634,"Attempt":0,"Launch Time":1427397542844,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542861,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":471411,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2638,"Index":635,"Attempt":0,"Launch Time":1427397542850,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542861,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":340115,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2640,"Index":637,"Attempt":0,"Launch Time":1427397542851,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542861,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":343197,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2644,"Index":641,"Attempt":0,"Launch Time":1427397542861,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2645,"Index":642,"Attempt":0,"Launch Time":1427397542867,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2631,"Index":628,"Attempt":0,"Launch Time":1427397542828,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542867,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":32,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":19043003,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2646,"Index":643,"Attempt":0,"Launch Time":1427397542868,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2642,"Index":639,"Attempt":0,"Launch Time":1427397542860,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542869,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":357211,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2647,"Index":644,"Attempt":0,"Launch Time":1427397542870,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2644,"Index":641,"Attempt":0,"Launch Time":1427397542861,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542870,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":358107,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2648,"Index":645,"Attempt":0,"Launch Time":1427397542871,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2641,"Index":638,"Attempt":0,"Launch Time":1427397542854,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542871,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":402907,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2649,"Index":646,"Attempt":0,"Launch Time":1427397542875,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2643,"Index":640,"Attempt":0,"Launch Time":1427397542860,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542875,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1248717,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2650,"Index":647,"Attempt":0,"Launch Time":1427397542878,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2646,"Index":643,"Attempt":0,"Launch Time":1427397542868,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542878,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":409759,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2651,"Index":648,"Attempt":0,"Launch Time":1427397542878,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2647,"Index":644,"Attempt":0,"Launch Time":1427397542870,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542879,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":625271,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2652,"Index":649,"Attempt":0,"Launch Time":1427397542880,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2648,"Index":645,"Attempt":0,"Launch Time":1427397542871,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542881,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":641811,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2653,"Index":650,"Attempt":0,"Launch Time":1427397542886,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2654,"Index":651,"Attempt":0,"Launch Time":1427397542886,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2655,"Index":652,"Attempt":0,"Launch Time":1427397542886,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2651,"Index":648,"Attempt":0,"Launch Time":1427397542878,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542887,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":547142,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2649,"Index":646,"Attempt":0,"Launch Time":1427397542875,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542887,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":664834,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2650,"Index":647,"Attempt":0,"Launch Time":1427397542878,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542887,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":379074,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2656,"Index":653,"Attempt":0,"Launch Time":1427397542890,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2632,"Index":629,"Attempt":0,"Launch Time":1427397542835,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542890,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":31,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":388731,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2657,"Index":654,"Attempt":0,"Launch Time":1427397542893,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2652,"Index":649,"Attempt":0,"Launch Time":1427397542880,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542893,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":344434,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2658,"Index":655,"Attempt":0,"Launch Time":1427397542893,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2654,"Index":651,"Attempt":0,"Launch Time":1427397542886,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542893,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":299160,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2659,"Index":656,"Attempt":0,"Launch Time":1427397542893,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2655,"Index":652,"Attempt":0,"Launch Time":1427397542886,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542893,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":327644,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2660,"Index":657,"Attempt":0,"Launch Time":1427397542894,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2653,"Index":650,"Attempt":0,"Launch Time":1427397542886,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542894,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":325935,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2661,"Index":658,"Attempt":0,"Launch Time":1427397542899,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2657,"Index":654,"Attempt":0,"Launch Time":1427397542893,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542899,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":307370,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2662,"Index":659,"Attempt":0,"Launch Time":1427397542900,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2658,"Index":655,"Attempt":0,"Launch Time":1427397542893,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542900,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":308555,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2663,"Index":660,"Attempt":0,"Launch Time":1427397542902,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2660,"Index":657,"Attempt":0,"Launch Time":1427397542894,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542902,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":508307,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2664,"Index":661,"Attempt":0,"Launch Time":1427397542906,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2661,"Index":658,"Attempt":0,"Launch Time":1427397542899,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542906,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":313634,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2665,"Index":662,"Attempt":0,"Launch Time":1427397542908,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2663,"Index":660,"Attempt":0,"Launch Time":1427397542902,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542909,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":308580,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2666,"Index":663,"Attempt":0,"Launch Time":1427397542911,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2659,"Index":656,"Attempt":0,"Launch Time":1427397542893,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542911,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":334657,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2667,"Index":664,"Attempt":0,"Launch Time":1427397542912,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2664,"Index":661,"Attempt":0,"Launch Time":1427397542906,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542912,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":292517,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2668,"Index":665,"Attempt":0,"Launch Time":1427397542917,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2662,"Index":659,"Attempt":0,"Launch Time":1427397542900,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542917,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":344633,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2669,"Index":666,"Attempt":0,"Launch Time":1427397542917,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2665,"Index":662,"Attempt":0,"Launch Time":1427397542908,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542917,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":375855,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2670,"Index":667,"Attempt":0,"Launch Time":1427397542919,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2667,"Index":664,"Attempt":0,"Launch Time":1427397542912,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542919,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":351353,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2671,"Index":668,"Attempt":0,"Launch Time":1427397542920,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2666,"Index":663,"Attempt":0,"Launch Time":1427397542911,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542920,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":338646,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2672,"Index":669,"Attempt":0,"Launch Time":1427397542926,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2669,"Index":666,"Attempt":0,"Launch Time":1427397542917,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542926,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":379100,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2673,"Index":670,"Attempt":0,"Launch Time":1427397542927,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2674,"Index":671,"Attempt":0,"Launch Time":1427397542927,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2668,"Index":665,"Attempt":0,"Launch Time":1427397542917,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542927,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":710441,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2670,"Index":667,"Attempt":0,"Launch Time":1427397542919,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542927,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":394617,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2675,"Index":672,"Attempt":0,"Launch Time":1427397542929,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2671,"Index":668,"Attempt":0,"Launch Time":1427397542920,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542929,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":461799,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2676,"Index":673,"Attempt":0,"Launch Time":1427397542934,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2672,"Index":669,"Attempt":0,"Launch Time":1427397542926,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542935,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":361458,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2677,"Index":674,"Attempt":0,"Launch Time":1427397542935,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2674,"Index":671,"Attempt":0,"Launch Time":1427397542927,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542936,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":356693,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2678,"Index":675,"Attempt":0,"Launch Time":1427397542936,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2673,"Index":670,"Attempt":0,"Launch Time":1427397542927,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542936,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":351481,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2679,"Index":676,"Attempt":0,"Launch Time":1427397542937,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2675,"Index":672,"Attempt":0,"Launch Time":1427397542929,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542937,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":339685,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2680,"Index":677,"Attempt":0,"Launch Time":1427397542954,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2676,"Index":673,"Attempt":0,"Launch Time":1427397542934,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542954,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":401946,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2681,"Index":678,"Attempt":0,"Launch Time":1427397542954,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2682,"Index":679,"Attempt":0,"Launch Time":1427397542954,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2656,"Index":653,"Attempt":0,"Launch Time":1427397542890,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542954,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":25,"Executor Run Time":26,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":377355,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2677,"Index":674,"Attempt":0,"Launch Time":1427397542935,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542955,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":359015,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2683,"Index":680,"Attempt":0,"Launch Time":1427397542960,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2679,"Index":676,"Attempt":0,"Launch Time":1427397542937,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542961,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":11,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":444455,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2684,"Index":681,"Attempt":0,"Launch Time":1427397542968,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2634,"Index":631,"Attempt":0,"Launch Time":1427397542841,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542969,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":37,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":356384,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2685,"Index":682,"Attempt":0,"Launch Time":1427397542969,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2645,"Index":642,"Attempt":0,"Launch Time":1427397542867,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542969,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":23,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":352083,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2686,"Index":683,"Attempt":0,"Launch Time":1427397542971,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2635,"Index":632,"Attempt":0,"Launch Time":1427397542842,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542971,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":16,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":389717,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2687,"Index":684,"Attempt":0,"Launch Time":1427397542977,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2685,"Index":682,"Attempt":0,"Launch Time":1427397542969,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542977,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":333883,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2688,"Index":685,"Attempt":0,"Launch Time":1427397542977,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2684,"Index":681,"Attempt":0,"Launch Time":1427397542968,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542977,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":403015,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2689,"Index":686,"Attempt":0,"Launch Time":1427397542979,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2686,"Index":683,"Attempt":0,"Launch Time":1427397542971,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542979,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":406198,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2690,"Index":687,"Attempt":0,"Launch Time":1427397542981,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2681,"Index":678,"Attempt":0,"Launch Time":1427397542954,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542981,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":831211,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2691,"Index":688,"Attempt":0,"Launch Time":1427397542985,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2687,"Index":684,"Attempt":0,"Launch Time":1427397542977,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542985,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":359400,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2692,"Index":689,"Attempt":0,"Launch Time":1427397542985,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2688,"Index":685,"Attempt":0,"Launch Time":1427397542977,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542985,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":364672,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2693,"Index":690,"Attempt":0,"Launch Time":1427397542986,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2689,"Index":686,"Attempt":0,"Launch Time":1427397542979,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542986,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":332041,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2694,"Index":691,"Attempt":0,"Launch Time":1427397542988,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2690,"Index":687,"Attempt":0,"Launch Time":1427397542981,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542989,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":310412,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2691,"Index":688,"Attempt":0,"Launch Time":1427397542985,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542991,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":320265,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2695,"Index":692,"Attempt":0,"Launch Time":1427397542992,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2696,"Index":693,"Attempt":0,"Launch Time":1427397542992,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2692,"Index":689,"Attempt":0,"Launch Time":1427397542985,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542993,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":348531,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2697,"Index":694,"Attempt":0,"Launch Time":1427397542993,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2693,"Index":690,"Attempt":0,"Launch Time":1427397542986,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542993,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":341172,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2698,"Index":695,"Attempt":0,"Launch Time":1427397543000,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2695,"Index":692,"Attempt":0,"Launch Time":1427397542992,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543000,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":367496,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2699,"Index":696,"Attempt":0,"Launch Time":1427397543001,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2700,"Index":697,"Attempt":0,"Launch Time":1427397543001,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2696,"Index":693,"Attempt":0,"Launch Time":1427397542992,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543002,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":412478,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2697,"Index":694,"Attempt":0,"Launch Time":1427397542993,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543002,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":425714,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2701,"Index":698,"Attempt":0,"Launch Time":1427397543007,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2698,"Index":695,"Attempt":0,"Launch Time":1427397543000,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543007,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":317005,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2702,"Index":699,"Attempt":0,"Launch Time":1427397543009,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2699,"Index":696,"Attempt":0,"Launch Time":1427397543001,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543009,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":329732,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2703,"Index":700,"Attempt":0,"Launch Time":1427397543010,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2700,"Index":697,"Attempt":0,"Launch Time":1427397543001,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543010,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9612,"Shuffle Write Time":591151,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2704,"Index":701,"Attempt":0,"Launch Time":1427397543013,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2701,"Index":698,"Attempt":0,"Launch Time":1427397543007,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543014,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":308157,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2705,"Index":702,"Attempt":0,"Launch Time":1427397543017,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2703,"Index":700,"Attempt":0,"Launch Time":1427397543010,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543017,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":333647,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2706,"Index":703,"Attempt":0,"Launch Time":1427397543020,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2694,"Index":691,"Attempt":0,"Launch Time":1427397542988,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543020,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":27,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":413483,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2707,"Index":704,"Attempt":0,"Launch Time":1427397543021,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2704,"Index":701,"Attempt":0,"Launch Time":1427397543013,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543021,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":354248,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2708,"Index":705,"Attempt":0,"Launch Time":1427397543025,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2702,"Index":699,"Attempt":0,"Launch Time":1427397543009,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543025,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":651728,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2709,"Index":706,"Attempt":0,"Launch Time":1427397543026,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2705,"Index":702,"Attempt":0,"Launch Time":1427397543017,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543026,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":695161,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2710,"Index":707,"Attempt":0,"Launch Time":1427397543031,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2707,"Index":704,"Attempt":0,"Launch Time":1427397543021,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543031,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":370043,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2711,"Index":708,"Attempt":0,"Launch Time":1427397543033,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2708,"Index":705,"Attempt":0,"Launch Time":1427397543025,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543033,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9615,"Shuffle Write Time":418149,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2712,"Index":709,"Attempt":0,"Launch Time":1427397543035,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2709,"Index":706,"Attempt":0,"Launch Time":1427397543026,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543035,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":393559,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2713,"Index":710,"Attempt":0,"Launch Time":1427397543036,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2706,"Index":703,"Attempt":0,"Launch Time":1427397543020,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543036,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":2020827,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2714,"Index":711,"Attempt":0,"Launch Time":1427397543039,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2710,"Index":707,"Attempt":0,"Launch Time":1427397543031,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543039,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":382803,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2715,"Index":712,"Attempt":0,"Launch Time":1427397543041,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2711,"Index":708,"Attempt":0,"Launch Time":1427397543033,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543041,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":412911,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2716,"Index":713,"Attempt":0,"Launch Time":1427397543043,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2712,"Index":709,"Attempt":0,"Launch Time":1427397543035,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543043,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":343773,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2717,"Index":714,"Attempt":0,"Launch Time":1427397543048,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2714,"Index":711,"Attempt":0,"Launch Time":1427397543039,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543048,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":367958,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2718,"Index":715,"Attempt":0,"Launch Time":1427397543049,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2715,"Index":712,"Attempt":0,"Launch Time":1427397543041,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543049,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":357978,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2719,"Index":716,"Attempt":0,"Launch Time":1427397543050,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2716,"Index":713,"Attempt":0,"Launch Time":1427397543043,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543050,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9613,"Shuffle Write Time":341089,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2720,"Index":717,"Attempt":0,"Launch Time":1427397543056,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2717,"Index":714,"Attempt":0,"Launch Time":1427397543048,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543056,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":342411,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2721,"Index":718,"Attempt":0,"Launch Time":1427397543058,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2718,"Index":715,"Attempt":0,"Launch Time":1427397543049,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543058,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":345892,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2722,"Index":719,"Attempt":0,"Launch Time":1427397543058,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2719,"Index":716,"Attempt":0,"Launch Time":1427397543050,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543058,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":327708,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2723,"Index":720,"Attempt":0,"Launch Time":1427397543085,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2722,"Index":719,"Attempt":0,"Launch Time":1427397543058,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543085,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":26,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":371059,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2724,"Index":721,"Attempt":0,"Launch Time":1427397543085,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2720,"Index":717,"Attempt":0,"Launch Time":1427397543056,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543085,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":27,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":398612,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2725,"Index":722,"Attempt":0,"Launch Time":1427397543086,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2713,"Index":710,"Attempt":0,"Launch Time":1427397543036,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543086,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":49,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":669486,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2726,"Index":723,"Attempt":0,"Launch Time":1427397543092,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2724,"Index":721,"Attempt":0,"Launch Time":1427397543085,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543092,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":322890,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2727,"Index":724,"Attempt":0,"Launch Time":1427397543093,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2725,"Index":722,"Attempt":0,"Launch Time":1427397543086,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543093,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":322808,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2728,"Index":725,"Attempt":0,"Launch Time":1427397543096,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2683,"Index":680,"Attempt":0,"Launch Time":1427397542960,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543096,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":27,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":391439,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2729,"Index":726,"Attempt":0,"Launch Time":1427397543103,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2680,"Index":677,"Attempt":0,"Launch Time":1427397542954,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543103,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":149,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":437672,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2730,"Index":727,"Attempt":0,"Launch Time":1427397543103,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2728,"Index":725,"Attempt":0,"Launch Time":1427397543096,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543103,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":343220,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2731,"Index":728,"Attempt":0,"Launch Time":1427397543108,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2682,"Index":679,"Attempt":0,"Launch Time":1427397542954,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543109,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":147,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":293278,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2732,"Index":729,"Attempt":0,"Launch Time":1427397543110,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2729,"Index":726,"Attempt":0,"Launch Time":1427397543103,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543110,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":333976,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2733,"Index":730,"Attempt":0,"Launch Time":1427397543112,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2730,"Index":727,"Attempt":0,"Launch Time":1427397543103,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543113,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":328848,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2734,"Index":731,"Attempt":0,"Launch Time":1427397543118,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2732,"Index":729,"Attempt":0,"Launch Time":1427397543110,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543118,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":336057,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2735,"Index":732,"Attempt":0,"Launch Time":1427397543120,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2736,"Index":733,"Attempt":0,"Launch Time":1427397543121,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2733,"Index":730,"Attempt":0,"Launch Time":1427397543112,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543121,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":323669,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2731,"Index":728,"Attempt":0,"Launch Time":1427397543108,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543121,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":446819,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2737,"Index":734,"Attempt":0,"Launch Time":1427397543125,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2734,"Index":731,"Attempt":0,"Launch Time":1427397543118,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543125,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":417926,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2738,"Index":735,"Attempt":0,"Launch Time":1427397543128,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2736,"Index":733,"Attempt":0,"Launch Time":1427397543121,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543128,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":340020,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2739,"Index":736,"Attempt":0,"Launch Time":1427397543129,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2735,"Index":732,"Attempt":0,"Launch Time":1427397543120,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543129,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":434321,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2740,"Index":737,"Attempt":0,"Launch Time":1427397543132,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2737,"Index":734,"Attempt":0,"Launch Time":1427397543125,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543133,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":371739,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2741,"Index":738,"Attempt":0,"Launch Time":1427397543136,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2742,"Index":739,"Attempt":0,"Launch Time":1427397543136,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2738,"Index":735,"Attempt":0,"Launch Time":1427397543128,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543136,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":329470,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2727,"Index":724,"Attempt":0,"Launch Time":1427397543093,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543136,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":444817,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2743,"Index":740,"Attempt":0,"Launch Time":1427397543138,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2739,"Index":736,"Attempt":0,"Launch Time":1427397543129,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543138,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":372197,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2744,"Index":741,"Attempt":0,"Launch Time":1427397543139,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2740,"Index":737,"Attempt":0,"Launch Time":1427397543132,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543139,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":309938,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2745,"Index":742,"Attempt":0,"Launch Time":1427397543143,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2741,"Index":738,"Attempt":0,"Launch Time":1427397543136,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543143,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":332691,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2746,"Index":743,"Attempt":0,"Launch Time":1427397543143,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2742,"Index":739,"Attempt":0,"Launch Time":1427397543136,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543143,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":299311,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2747,"Index":744,"Attempt":0,"Launch Time":1427397543146,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2743,"Index":740,"Attempt":0,"Launch Time":1427397543138,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543146,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":331874,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2748,"Index":745,"Attempt":0,"Launch Time":1427397543147,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2744,"Index":741,"Attempt":0,"Launch Time":1427397543139,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543147,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":362624,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2749,"Index":746,"Attempt":0,"Launch Time":1427397543153,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2746,"Index":743,"Attempt":0,"Launch Time":1427397543143,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543153,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":341725,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2750,"Index":747,"Attempt":0,"Launch Time":1427397543155,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2747,"Index":744,"Attempt":0,"Launch Time":1427397543146,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543155,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":360229,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2751,"Index":748,"Attempt":0,"Launch Time":1427397543156,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2748,"Index":745,"Attempt":0,"Launch Time":1427397543147,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543156,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":330190,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2752,"Index":749,"Attempt":0,"Launch Time":1427397543159,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2745,"Index":742,"Attempt":0,"Launch Time":1427397543143,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543159,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":352674,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2753,"Index":750,"Attempt":0,"Launch Time":1427397543161,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2749,"Index":746,"Attempt":0,"Launch Time":1427397543153,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543161,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":359420,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2754,"Index":751,"Attempt":0,"Launch Time":1427397543162,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2750,"Index":747,"Attempt":0,"Launch Time":1427397543155,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543162,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":335041,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2755,"Index":752,"Attempt":0,"Launch Time":1427397543163,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2751,"Index":748,"Attempt":0,"Launch Time":1427397543156,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543163,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":330655,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2756,"Index":753,"Attempt":0,"Launch Time":1427397543169,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2753,"Index":750,"Attempt":0,"Launch Time":1427397543161,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543169,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":623863,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2757,"Index":754,"Attempt":0,"Launch Time":1427397543169,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2752,"Index":749,"Attempt":0,"Launch Time":1427397543159,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543169,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":315302,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2758,"Index":755,"Attempt":0,"Launch Time":1427397543170,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2755,"Index":752,"Attempt":0,"Launch Time":1427397543163,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543170,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":327839,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2759,"Index":756,"Attempt":0,"Launch Time":1427397543173,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2754,"Index":751,"Attempt":0,"Launch Time":1427397543162,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543174,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":371610,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2760,"Index":757,"Attempt":0,"Launch Time":1427397543178,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2756,"Index":753,"Attempt":0,"Launch Time":1427397543169,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543178,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":378845,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2761,"Index":758,"Attempt":0,"Launch Time":1427397543178,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2757,"Index":754,"Attempt":0,"Launch Time":1427397543169,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543178,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":354349,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2762,"Index":759,"Attempt":0,"Launch Time":1427397543178,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2758,"Index":755,"Attempt":0,"Launch Time":1427397543170,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543178,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":354808,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2763,"Index":760,"Attempt":0,"Launch Time":1427397543181,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2759,"Index":756,"Attempt":0,"Launch Time":1427397543173,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543182,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":313403,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2764,"Index":761,"Attempt":0,"Launch Time":1427397543186,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2765,"Index":762,"Attempt":0,"Launch Time":1427397543186,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2762,"Index":759,"Attempt":0,"Launch Time":1427397543178,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543186,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":327180,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2760,"Index":757,"Attempt":0,"Launch Time":1427397543178,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543186,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":341794,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2766,"Index":763,"Attempt":0,"Launch Time":1427397543186,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2761,"Index":758,"Attempt":0,"Launch Time":1427397543178,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543187,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":451128,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2767,"Index":764,"Attempt":0,"Launch Time":1427397543189,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2763,"Index":760,"Attempt":0,"Launch Time":1427397543181,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543189,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":336969,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2768,"Index":765,"Attempt":0,"Launch Time":1427397543205,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2766,"Index":763,"Attempt":0,"Launch Time":1427397543186,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543206,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":336638,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2769,"Index":766,"Attempt":0,"Launch Time":1427397543206,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2764,"Index":761,"Attempt":0,"Launch Time":1427397543186,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543206,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":19,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":356389,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2770,"Index":767,"Attempt":0,"Launch Time":1427397543208,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2765,"Index":762,"Attempt":0,"Launch Time":1427397543186,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543208,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":345407,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2771,"Index":768,"Attempt":0,"Launch Time":1427397543210,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2726,"Index":723,"Attempt":0,"Launch Time":1427397543092,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543210,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":326984,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2772,"Index":769,"Attempt":0,"Launch Time":1427397543214,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2768,"Index":765,"Attempt":0,"Launch Time":1427397543205,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543214,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":344006,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2773,"Index":770,"Attempt":0,"Launch Time":1427397543217,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2678,"Index":675,"Attempt":0,"Launch Time":1427397542936,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543217,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":280,"Result Size":930,"JVM GC Time":23,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":419616,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2774,"Index":771,"Attempt":0,"Launch Time":1427397543218,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2723,"Index":720,"Attempt":0,"Launch Time":1427397543085,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543218,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":133,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":635551,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2775,"Index":772,"Attempt":0,"Launch Time":1427397543221,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2721,"Index":718,"Attempt":0,"Launch Time":1427397543058,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543221,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":163,"Result Size":930,"JVM GC Time":16,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":347006,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2776,"Index":773,"Attempt":0,"Launch Time":1427397543224,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2773,"Index":770,"Attempt":0,"Launch Time":1427397543217,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543225,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":446862,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2777,"Index":774,"Attempt":0,"Launch Time":1427397543226,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2774,"Index":771,"Attempt":0,"Launch Time":1427397543218,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543227,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":392666,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2778,"Index":775,"Attempt":0,"Launch Time":1427397543229,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2775,"Index":772,"Attempt":0,"Launch Time":1427397543221,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543229,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":344320,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2779,"Index":776,"Attempt":0,"Launch Time":1427397543232,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2776,"Index":773,"Attempt":0,"Launch Time":1427397543224,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543233,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":390960,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2780,"Index":777,"Attempt":0,"Launch Time":1427397543234,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2771,"Index":768,"Attempt":0,"Launch Time":1427397543210,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543234,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":383783,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2781,"Index":778,"Attempt":0,"Launch Time":1427397543236,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2777,"Index":774,"Attempt":0,"Launch Time":1427397543226,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543236,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":467336,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2782,"Index":779,"Attempt":0,"Launch Time":1427397543238,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2778,"Index":775,"Attempt":0,"Launch Time":1427397543229,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543238,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":331618,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2783,"Index":780,"Attempt":0,"Launch Time":1427397543240,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2779,"Index":776,"Attempt":0,"Launch Time":1427397543232,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543240,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":371000,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2784,"Index":781,"Attempt":0,"Launch Time":1427397543242,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2780,"Index":777,"Attempt":0,"Launch Time":1427397543234,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543242,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":350211,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2785,"Index":782,"Attempt":0,"Launch Time":1427397543248,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2783,"Index":780,"Attempt":0,"Launch Time":1427397543240,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543248,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":317142,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2786,"Index":783,"Attempt":0,"Launch Time":1427397543250,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2782,"Index":779,"Attempt":0,"Launch Time":1427397543238,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543250,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":358354,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2787,"Index":784,"Attempt":0,"Launch Time":1427397543250,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2784,"Index":781,"Attempt":0,"Launch Time":1427397543242,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543250,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":364565,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2788,"Index":785,"Attempt":0,"Launch Time":1427397543251,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2781,"Index":778,"Attempt":0,"Launch Time":1427397543236,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543251,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":529162,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2789,"Index":786,"Attempt":0,"Launch Time":1427397543255,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2785,"Index":782,"Attempt":0,"Launch Time":1427397543248,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543255,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":305615,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2790,"Index":787,"Attempt":0,"Launch Time":1427397543258,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2791,"Index":788,"Attempt":0,"Launch Time":1427397543258,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2786,"Index":783,"Attempt":0,"Launch Time":1427397543250,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543258,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":379148,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2787,"Index":784,"Attempt":0,"Launch Time":1427397543250,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543258,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":372179,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2792,"Index":789,"Attempt":0,"Launch Time":1427397543260,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2788,"Index":785,"Attempt":0,"Launch Time":1427397543251,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543260,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":396147,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2793,"Index":790,"Attempt":0,"Launch Time":1427397543262,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2789,"Index":786,"Attempt":0,"Launch Time":1427397543255,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543262,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":368901,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2794,"Index":791,"Attempt":0,"Launch Time":1427397543266,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2791,"Index":788,"Attempt":0,"Launch Time":1427397543258,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543266,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":353284,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2795,"Index":792,"Attempt":0,"Launch Time":1427397543266,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2770,"Index":767,"Attempt":0,"Launch Time":1427397543208,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543267,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":57,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":336829,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2796,"Index":793,"Attempt":0,"Launch Time":1427397543268,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2792,"Index":789,"Attempt":0,"Launch Time":1427397543260,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543268,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":330263,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2797,"Index":794,"Attempt":0,"Launch Time":1427397543272,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2798,"Index":795,"Attempt":0,"Launch Time":1427397543272,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2790,"Index":787,"Attempt":0,"Launch Time":1427397543258,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543272,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":355427,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2793,"Index":790,"Attempt":0,"Launch Time":1427397543262,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543273,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":341534,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2799,"Index":796,"Attempt":0,"Launch Time":1427397543273,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2794,"Index":791,"Attempt":0,"Launch Time":1427397543266,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543273,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":338448,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2800,"Index":797,"Attempt":0,"Launch Time":1427397543276,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2796,"Index":793,"Attempt":0,"Launch Time":1427397543268,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543276,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":336106,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2801,"Index":798,"Attempt":0,"Launch Time":1427397543279,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2798,"Index":795,"Attempt":0,"Launch Time":1427397543272,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543279,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":338610,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2802,"Index":799,"Attempt":0,"Launch Time":1427397543280,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2797,"Index":794,"Attempt":0,"Launch Time":1427397543272,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543280,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":321135,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2803,"Index":800,"Attempt":0,"Launch Time":1427397543280,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2795,"Index":792,"Attempt":0,"Launch Time":1427397543266,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543281,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":2855698,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2804,"Index":801,"Attempt":0,"Launch Time":1427397543281,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2799,"Index":796,"Attempt":0,"Launch Time":1427397543273,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543281,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":382919,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2805,"Index":802,"Attempt":0,"Launch Time":1427397543284,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2800,"Index":797,"Attempt":0,"Launch Time":1427397543276,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543284,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":362422,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2806,"Index":803,"Attempt":0,"Launch Time":1427397543286,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2807,"Index":804,"Attempt":0,"Launch Time":1427397543287,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2801,"Index":798,"Attempt":0,"Launch Time":1427397543279,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543287,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":322216,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2802,"Index":799,"Attempt":0,"Launch Time":1427397543280,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543287,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":326364,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2808,"Index":805,"Attempt":0,"Launch Time":1427397543288,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2803,"Index":800,"Attempt":0,"Launch Time":1427397543280,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543289,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":355695,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2809,"Index":806,"Attempt":0,"Launch Time":1427397543290,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2805,"Index":802,"Attempt":0,"Launch Time":1427397543284,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543291,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":284589,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2810,"Index":807,"Attempt":0,"Launch Time":1427397543293,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2804,"Index":801,"Attempt":0,"Launch Time":1427397543281,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543294,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":382608,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2811,"Index":808,"Attempt":0,"Launch Time":1427397543294,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2806,"Index":803,"Attempt":0,"Launch Time":1427397543286,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543295,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":381974,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2812,"Index":809,"Attempt":0,"Launch Time":1427397543295,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2807,"Index":804,"Attempt":0,"Launch Time":1427397543287,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543295,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":380653,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2813,"Index":810,"Attempt":0,"Launch Time":1427397543298,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2808,"Index":805,"Attempt":0,"Launch Time":1427397543288,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543298,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":349616,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2814,"Index":811,"Attempt":0,"Launch Time":1427397543312,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2811,"Index":808,"Attempt":0,"Launch Time":1427397543294,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543312,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":357933,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2815,"Index":812,"Attempt":0,"Launch Time":1427397543313,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2809,"Index":806,"Attempt":0,"Launch Time":1427397543290,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543313,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":22,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":689066,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2816,"Index":813,"Attempt":0,"Launch Time":1427397543313,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2812,"Index":809,"Attempt":0,"Launch Time":1427397543295,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543313,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":371722,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2817,"Index":814,"Attempt":0,"Launch Time":1427397543319,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2814,"Index":811,"Attempt":0,"Launch Time":1427397543312,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543319,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":329116,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2818,"Index":815,"Attempt":0,"Launch Time":1427397543320,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2816,"Index":813,"Attempt":0,"Launch Time":1427397543313,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543320,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":341965,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2819,"Index":816,"Attempt":0,"Launch Time":1427397543321,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2815,"Index":812,"Attempt":0,"Launch Time":1427397543313,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543322,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":320049,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2820,"Index":817,"Attempt":0,"Launch Time":1427397543326,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2817,"Index":814,"Attempt":0,"Launch Time":1427397543319,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543327,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":331005,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2821,"Index":818,"Attempt":0,"Launch Time":1427397543334,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2822,"Index":819,"Attempt":0,"Launch Time":1427397543334,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2767,"Index":764,"Attempt":0,"Launch Time":1427397543189,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543334,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":142,"Result Size":930,"JVM GC Time":19,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":380909,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2823,"Index":820,"Attempt":0,"Launch Time":1427397543335,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2824,"Index":821,"Attempt":0,"Launch Time":1427397543335,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2825,"Index":822,"Attempt":0,"Launch Time":1427397543335,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2769,"Index":766,"Attempt":0,"Launch Time":1427397543206,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543335,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":122,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":560598,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2810,"Index":807,"Attempt":0,"Launch Time":1427397543293,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543335,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":39,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":322446,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2819,"Index":816,"Attempt":0,"Launch Time":1427397543321,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543335,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":376961,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2818,"Index":815,"Attempt":0,"Launch Time":1427397543320,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543335,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":369599,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2826,"Index":823,"Attempt":0,"Launch Time":1427397543335,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2820,"Index":817,"Attempt":0,"Launch Time":1427397543326,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543336,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":414763,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2827,"Index":824,"Attempt":0,"Launch Time":1427397543341,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2821,"Index":818,"Attempt":0,"Launch Time":1427397543334,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543341,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":348013,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2823,"Index":820,"Attempt":0,"Launch Time":1427397543335,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543342,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":355959,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2828,"Index":825,"Attempt":0,"Launch Time":1427397543342,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2829,"Index":826,"Attempt":0,"Launch Time":1427397543346,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2826,"Index":823,"Attempt":0,"Launch Time":1427397543335,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543346,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":649904,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2830,"Index":827,"Attempt":0,"Launch Time":1427397543346,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2824,"Index":821,"Attempt":0,"Launch Time":1427397543335,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543347,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":358518,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2831,"Index":828,"Attempt":0,"Launch Time":1427397543349,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2832,"Index":829,"Attempt":0,"Launch Time":1427397543349,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2828,"Index":825,"Attempt":0,"Launch Time":1427397543342,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543349,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":297571,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2827,"Index":824,"Attempt":0,"Launch Time":1427397543341,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543349,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1079638,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2833,"Index":830,"Attempt":0,"Launch Time":1427397543353,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2829,"Index":826,"Attempt":0,"Launch Time":1427397543346,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543353,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":338454,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2834,"Index":831,"Attempt":0,"Launch Time":1427397543356,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2835,"Index":832,"Attempt":0,"Launch Time":1427397543357,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2830,"Index":827,"Attempt":0,"Launch Time":1427397543346,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543357,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":338912,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2832,"Index":829,"Attempt":0,"Launch Time":1427397543349,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543357,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":352560,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2836,"Index":833,"Attempt":0,"Launch Time":1427397543358,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2813,"Index":810,"Attempt":0,"Launch Time":1427397543298,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543358,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":9,"Executor Run Time":20,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":350603,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2837,"Index":834,"Attempt":0,"Launch Time":1427397543358,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2831,"Index":828,"Attempt":0,"Launch Time":1427397543349,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543358,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":384157,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2838,"Index":835,"Attempt":0,"Launch Time":1427397543360,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2833,"Index":830,"Attempt":0,"Launch Time":1427397543353,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543360,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":368434,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2839,"Index":836,"Attempt":0,"Launch Time":1427397543365,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2835,"Index":832,"Attempt":0,"Launch Time":1427397543357,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543366,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":441303,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2840,"Index":837,"Attempt":0,"Launch Time":1427397543366,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2834,"Index":831,"Attempt":0,"Launch Time":1427397543356,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543366,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":448575,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2841,"Index":838,"Attempt":0,"Launch Time":1427397543367,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2837,"Index":834,"Attempt":0,"Launch Time":1427397543358,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543367,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":415586,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2842,"Index":839,"Attempt":0,"Launch Time":1427397543369,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2838,"Index":835,"Attempt":0,"Launch Time":1427397543360,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543369,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":397984,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2843,"Index":840,"Attempt":0,"Launch Time":1427397543373,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2839,"Index":836,"Attempt":0,"Launch Time":1427397543365,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543373,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":325516,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2844,"Index":841,"Attempt":0,"Launch Time":1427397543373,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2840,"Index":837,"Attempt":0,"Launch Time":1427397543366,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543374,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":439084,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2845,"Index":842,"Attempt":0,"Launch Time":1427397543374,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2841,"Index":838,"Attempt":0,"Launch Time":1427397543367,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543374,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":340591,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2846,"Index":843,"Attempt":0,"Launch Time":1427397543377,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2842,"Index":839,"Attempt":0,"Launch Time":1427397543369,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543377,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":301374,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2847,"Index":844,"Attempt":0,"Launch Time":1427397543380,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2843,"Index":840,"Attempt":0,"Launch Time":1427397543373,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543380,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":347677,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2848,"Index":845,"Attempt":0,"Launch Time":1427397543381,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2845,"Index":842,"Attempt":0,"Launch Time":1427397543374,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543381,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":351820,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2849,"Index":846,"Attempt":0,"Launch Time":1427397543381,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543389,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2844,"Index":841,"Attempt":0,"Launch Time":1427397543373,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543381,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":348949,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2850,"Index":847,"Attempt":0,"Launch Time":1427397543388,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2847,"Index":844,"Attempt":0,"Launch Time":1427397543380,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543388,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":427393,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2851,"Index":848,"Attempt":0,"Launch Time":1427397543389,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2852,"Index":849,"Attempt":0,"Launch Time":1427397543389,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2848,"Index":845,"Attempt":0,"Launch Time":1427397543381,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543389,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":431542,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2849,"Index":846,"Attempt":0,"Launch Time":1427397543381,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543389,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":423444,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2853,"Index":850,"Attempt":0,"Launch Time":1427397543389,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2846,"Index":843,"Attempt":0,"Launch Time":1427397543377,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543397,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":409299,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2854,"Index":851,"Attempt":0,"Launch Time":1427397543397,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2772,"Index":769,"Attempt":0,"Launch Time":1427397543214,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543397,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":40,"Executor Run Time":29,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":314723,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2855,"Index":852,"Attempt":0,"Launch Time":1427397543398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2850,"Index":847,"Attempt":0,"Launch Time":1427397543388,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543398,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":326677,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2856,"Index":853,"Attempt":0,"Launch Time":1427397543398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2836,"Index":833,"Attempt":0,"Launch Time":1427397543358,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543398,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":20,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":410705,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2857,"Index":854,"Attempt":0,"Launch Time":1427397543405,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2854,"Index":851,"Attempt":0,"Launch Time":1427397543397,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543405,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":398644,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2858,"Index":855,"Attempt":0,"Launch Time":1427397543405,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2851,"Index":848,"Attempt":0,"Launch Time":1427397543389,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543406,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":9,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":367832,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2859,"Index":856,"Attempt":0,"Launch Time":1427397543406,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2852,"Index":849,"Attempt":0,"Launch Time":1427397543389,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543407,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":425841,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2860,"Index":857,"Attempt":0,"Launch Time":1427397543408,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2856,"Index":853,"Attempt":0,"Launch Time":1427397543398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543408,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":392767,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2861,"Index":858,"Attempt":0,"Launch Time":1427397543423,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543430,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2858,"Index":855,"Attempt":0,"Launch Time":1427397543405,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543424,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":283715,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2862,"Index":859,"Attempt":0,"Launch Time":1427397543427,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543434,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2855,"Index":852,"Attempt":0,"Launch Time":1427397543398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543427,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":21,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":364830,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2863,"Index":860,"Attempt":0,"Launch Time":1427397543429,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543436,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2853,"Index":850,"Attempt":0,"Launch Time":1427397543389,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543429,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":25,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":333717,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2864,"Index":861,"Attempt":0,"Launch Time":1427397543430,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543437,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2861,"Index":858,"Attempt":0,"Launch Time":1427397543423,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543430,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":324221,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2865,"Index":862,"Attempt":0,"Launch Time":1427397543432,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543438,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2822,"Index":819,"Attempt":0,"Launch Time":1427397543334,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543432,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":95,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":348251,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2866,"Index":863,"Attempt":0,"Launch Time":1427397543434,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543446,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2862,"Index":859,"Attempt":0,"Launch Time":1427397543427,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543434,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":329992,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2867,"Index":864,"Attempt":0,"Launch Time":1427397543436,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543447,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2863,"Index":860,"Attempt":0,"Launch Time":1427397543429,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543436,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":309915,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2868,"Index":865,"Attempt":0,"Launch Time":1427397543437,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543444,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2864,"Index":861,"Attempt":0,"Launch Time":1427397543430,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543437,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":311270,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2869,"Index":866,"Attempt":0,"Launch Time":1427397543438,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543447,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2865,"Index":862,"Attempt":0,"Launch Time":1427397543432,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543438,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":311851,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2870,"Index":867,"Attempt":0,"Launch Time":1427397543444,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543451,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2868,"Index":865,"Attempt":0,"Launch Time":1427397543437,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543444,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":340420,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2871,"Index":868,"Attempt":0,"Launch Time":1427397543446,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543453,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2866,"Index":863,"Attempt":0,"Launch Time":1427397543434,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543446,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":311949,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2872,"Index":869,"Attempt":0,"Launch Time":1427397543447,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2869,"Index":866,"Attempt":0,"Launch Time":1427397543438,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543447,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":332364,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2873,"Index":870,"Attempt":0,"Launch Time":1427397543447,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543454,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2867,"Index":864,"Attempt":0,"Launch Time":1427397543436,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543447,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":438391,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2874,"Index":871,"Attempt":0,"Launch Time":1427397543450,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2870,"Index":867,"Attempt":0,"Launch Time":1427397543444,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543451,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":496793,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2875,"Index":872,"Attempt":0,"Launch Time":1427397543453,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2871,"Index":868,"Attempt":0,"Launch Time":1427397543446,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543453,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":379938,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2876,"Index":873,"Attempt":0,"Launch Time":1427397543454,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2873,"Index":870,"Attempt":0,"Launch Time":1427397543447,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543454,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":354390,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2877,"Index":874,"Attempt":0,"Launch Time":1427397543460,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2874,"Index":871,"Attempt":0,"Launch Time":1427397543450,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543461,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":473061,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2878,"Index":875,"Attempt":0,"Launch Time":1427397543462,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2859,"Index":856,"Attempt":0,"Launch Time":1427397543406,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543462,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":55,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":498948,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2825,"Index":822,"Attempt":0,"Launch Time":1427397543335,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543494,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":130,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":376945,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2879,"Index":876,"Attempt":0,"Launch Time":1427397543495,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2880,"Index":877,"Attempt":0,"Launch Time":1427397543495,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2881,"Index":878,"Attempt":0,"Launch Time":1427397543495,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2882,"Index":879,"Attempt":0,"Launch Time":1427397543496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2883,"Index":880,"Attempt":0,"Launch Time":1427397543496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2884,"Index":881,"Attempt":0,"Launch Time":1427397543496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2876,"Index":873,"Attempt":0,"Launch Time":1427397543454,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543496,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":40,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":468133,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2860,"Index":857,"Attempt":0,"Launch Time":1427397543408,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543497,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":47,"Executor Run Time":40,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":352729,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2857,"Index":854,"Attempt":0,"Launch Time":1427397543405,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543497,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":64,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":452703,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2875,"Index":872,"Attempt":0,"Launch Time":1427397543453,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543498,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":414588,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2872,"Index":869,"Attempt":0,"Launch Time":1427397543447,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543498,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":29,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":400218,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2885,"Index":882,"Attempt":0,"Launch Time":1427397543501,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2878,"Index":875,"Attempt":0,"Launch Time":1427397543462,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543501,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":31,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":352195,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2886,"Index":883,"Attempt":0,"Launch Time":1427397543504,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2884,"Index":881,"Attempt":0,"Launch Time":1427397543496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543504,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":330226,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2887,"Index":884,"Attempt":0,"Launch Time":1427397543506,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2882,"Index":879,"Attempt":0,"Launch Time":1427397543496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543507,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":346008,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2888,"Index":885,"Attempt":0,"Launch Time":1427397543508,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2881,"Index":878,"Attempt":0,"Launch Time":1427397543495,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543508,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":327212,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2889,"Index":886,"Attempt":0,"Launch Time":1427397543509,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2883,"Index":880,"Attempt":0,"Launch Time":1427397543496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543509,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":367722,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2890,"Index":887,"Attempt":0,"Launch Time":1427397543509,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2877,"Index":874,"Attempt":0,"Launch Time":1427397543460,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543509,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":48,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":665763,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2891,"Index":888,"Attempt":0,"Launch Time":1427397543515,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2886,"Index":883,"Attempt":0,"Launch Time":1427397543504,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543515,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":437322,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2892,"Index":889,"Attempt":0,"Launch Time":1427397543517,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2889,"Index":886,"Attempt":0,"Launch Time":1427397543509,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543517,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":399308,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2893,"Index":890,"Attempt":0,"Launch Time":1427397543518,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2887,"Index":884,"Attempt":0,"Launch Time":1427397543506,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543518,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":496498,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2894,"Index":891,"Attempt":0,"Launch Time":1427397543523,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2891,"Index":888,"Attempt":0,"Launch Time":1427397543515,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543523,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":310674,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2895,"Index":892,"Attempt":0,"Launch Time":1427397543524,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2893,"Index":890,"Attempt":0,"Launch Time":1427397543518,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543524,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":318786,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2896,"Index":893,"Attempt":0,"Launch Time":1427397543528,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2888,"Index":885,"Attempt":0,"Launch Time":1427397543508,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543528,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":420784,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2897,"Index":894,"Attempt":0,"Launch Time":1427397543530,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2894,"Index":891,"Attempt":0,"Launch Time":1427397543523,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543530,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":313552,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2898,"Index":895,"Attempt":0,"Launch Time":1427397543531,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2892,"Index":889,"Attempt":0,"Launch Time":1427397543517,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543531,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":2980201,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2899,"Index":896,"Attempt":0,"Launch Time":1427397543532,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2895,"Index":892,"Attempt":0,"Launch Time":1427397543524,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543532,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":322500,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2900,"Index":897,"Attempt":0,"Launch Time":1427397543537,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2897,"Index":894,"Attempt":0,"Launch Time":1427397543530,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543537,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":407322,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2901,"Index":898,"Attempt":0,"Launch Time":1427397543540,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2898,"Index":895,"Attempt":0,"Launch Time":1427397543531,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543540,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":392735,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2902,"Index":899,"Attempt":0,"Launch Time":1427397543545,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2900,"Index":897,"Attempt":0,"Launch Time":1427397543537,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543545,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":372246,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2903,"Index":900,"Attempt":0,"Launch Time":1427397543547,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2901,"Index":898,"Attempt":0,"Launch Time":1427397543540,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543548,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":333260,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2904,"Index":901,"Attempt":0,"Launch Time":1427397543549,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2899,"Index":896,"Attempt":0,"Launch Time":1427397543532,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543549,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":349974,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2905,"Index":902,"Attempt":0,"Launch Time":1427397543550,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2896,"Index":893,"Attempt":0,"Launch Time":1427397543528,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543551,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":19,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":601961,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2906,"Index":903,"Attempt":0,"Launch Time":1427397543552,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2902,"Index":899,"Attempt":0,"Launch Time":1427397543545,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543552,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":317689,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2907,"Index":904,"Attempt":0,"Launch Time":1427397543572,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2906,"Index":903,"Attempt":0,"Launch Time":1427397543552,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543572,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":20,"Result Size":930,"JVM GC Time":12,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9608,"Shuffle Write Time":397809,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2908,"Index":905,"Attempt":0,"Launch Time":1427397543573,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2905,"Index":902,"Attempt":0,"Launch Time":1427397543550,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543573,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":20,"Result Size":930,"JVM GC Time":12,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":353556,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2909,"Index":906,"Attempt":0,"Launch Time":1427397543580,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2907,"Index":904,"Attempt":0,"Launch Time":1427397543572,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543580,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":434945,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2910,"Index":907,"Attempt":0,"Launch Time":1427397543583,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2908,"Index":905,"Attempt":0,"Launch Time":1427397543573,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543583,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":436567,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2911,"Index":908,"Attempt":0,"Launch Time":1427397543588,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2909,"Index":906,"Attempt":0,"Launch Time":1427397543580,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543588,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":318944,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2912,"Index":909,"Attempt":0,"Launch Time":1427397543589,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2910,"Index":907,"Attempt":0,"Launch Time":1427397543583,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543590,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":318107,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2913,"Index":910,"Attempt":0,"Launch Time":1427397543596,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2879,"Index":876,"Attempt":0,"Launch Time":1427397543495,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543596,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":373938,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2914,"Index":911,"Attempt":0,"Launch Time":1427397543597,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2880,"Index":877,"Attempt":0,"Launch Time":1427397543495,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543597,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":373769,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2915,"Index":912,"Attempt":0,"Launch Time":1427397543599,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2890,"Index":887,"Attempt":0,"Launch Time":1427397543509,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543599,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":319283,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2916,"Index":913,"Attempt":0,"Launch Time":1427397543599,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2885,"Index":882,"Attempt":0,"Launch Time":1427397543501,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543599,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":876044,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2917,"Index":914,"Attempt":0,"Launch Time":1427397543602,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2913,"Index":910,"Attempt":0,"Launch Time":1427397543596,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543602,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":302282,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2918,"Index":915,"Attempt":0,"Launch Time":1427397543604,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2914,"Index":911,"Attempt":0,"Launch Time":1427397543597,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543604,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":349184,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2919,"Index":916,"Attempt":0,"Launch Time":1427397543607,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2920,"Index":917,"Attempt":0,"Launch Time":1427397543607,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2916,"Index":913,"Attempt":0,"Launch Time":1427397543599,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543607,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":337392,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2915,"Index":912,"Attempt":0,"Launch Time":1427397543599,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543607,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":341718,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2917,"Index":914,"Attempt":0,"Launch Time":1427397543602,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543610,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":335840,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2921,"Index":918,"Attempt":0,"Launch Time":1427397543610,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2922,"Index":919,"Attempt":0,"Launch Time":1427397543613,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2918,"Index":915,"Attempt":0,"Launch Time":1427397543604,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543613,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":320542,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2923,"Index":920,"Attempt":0,"Launch Time":1427397543616,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2919,"Index":916,"Attempt":0,"Launch Time":1427397543607,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543616,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":403885,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2924,"Index":921,"Attempt":0,"Launch Time":1427397543617,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2920,"Index":917,"Attempt":0,"Launch Time":1427397543607,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543617,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":704507,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2925,"Index":922,"Attempt":0,"Launch Time":1427397543617,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2921,"Index":918,"Attempt":0,"Launch Time":1427397543610,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543618,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":349871,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2926,"Index":923,"Attempt":0,"Launch Time":1427397543621,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2922,"Index":919,"Attempt":0,"Launch Time":1427397543613,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543621,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":324321,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2927,"Index":924,"Attempt":0,"Launch Time":1427397543623,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2923,"Index":920,"Attempt":0,"Launch Time":1427397543616,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543624,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":347530,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2928,"Index":925,"Attempt":0,"Launch Time":1427397543625,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2925,"Index":922,"Attempt":0,"Launch Time":1427397543617,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543625,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":346997,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2929,"Index":926,"Attempt":0,"Launch Time":1427397543625,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2924,"Index":921,"Attempt":0,"Launch Time":1427397543617,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543625,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":325576,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2930,"Index":927,"Attempt":0,"Launch Time":1427397543630,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2926,"Index":923,"Attempt":0,"Launch Time":1427397543621,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543630,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":412153,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2931,"Index":928,"Attempt":0,"Launch Time":1427397543632,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2929,"Index":926,"Attempt":0,"Launch Time":1427397543625,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543633,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":323315,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2932,"Index":929,"Attempt":0,"Launch Time":1427397543633,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2927,"Index":924,"Attempt":0,"Launch Time":1427397543623,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543633,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":919364,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2933,"Index":930,"Attempt":0,"Launch Time":1427397543637,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2930,"Index":927,"Attempt":0,"Launch Time":1427397543630,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543637,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":335546,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2934,"Index":931,"Attempt":0,"Launch Time":1427397543637,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2928,"Index":925,"Attempt":0,"Launch Time":1427397543625,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543638,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":354140,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2935,"Index":932,"Attempt":0,"Launch Time":1427397543639,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2931,"Index":928,"Attempt":0,"Launch Time":1427397543632,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543639,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":372101,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2936,"Index":933,"Attempt":0,"Launch Time":1427397543642,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2932,"Index":929,"Attempt":0,"Launch Time":1427397543633,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543643,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":338468,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2937,"Index":934,"Attempt":0,"Launch Time":1427397543645,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2933,"Index":930,"Attempt":0,"Launch Time":1427397543637,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543645,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":358501,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2938,"Index":935,"Attempt":0,"Launch Time":1427397543645,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2934,"Index":931,"Attempt":0,"Launch Time":1427397543637,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543645,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":466896,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2939,"Index":936,"Attempt":0,"Launch Time":1427397543646,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2935,"Index":932,"Attempt":0,"Launch Time":1427397543639,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543646,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":330654,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2940,"Index":937,"Attempt":0,"Launch Time":1427397543649,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2936,"Index":933,"Attempt":0,"Launch Time":1427397543642,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543650,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":317831,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2941,"Index":938,"Attempt":0,"Launch Time":1427397543652,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2938,"Index":935,"Attempt":0,"Launch Time":1427397543645,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543652,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":322571,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2942,"Index":939,"Attempt":0,"Launch Time":1427397543653,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2939,"Index":936,"Attempt":0,"Launch Time":1427397543646,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543653,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":350233,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2943,"Index":940,"Attempt":0,"Launch Time":1427397543656,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2937,"Index":934,"Attempt":0,"Launch Time":1427397543645,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543656,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":817315,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2944,"Index":941,"Attempt":0,"Launch Time":1427397543657,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2940,"Index":937,"Attempt":0,"Launch Time":1427397543649,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543657,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":384618,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2945,"Index":942,"Attempt":0,"Launch Time":1427397543658,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2904,"Index":901,"Attempt":0,"Launch Time":1427397543549,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543659,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":107,"Result Size":930,"JVM GC Time":12,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":4345747,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2946,"Index":943,"Attempt":0,"Launch Time":1427397543660,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2947,"Index":944,"Attempt":0,"Launch Time":1427397543661,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2942,"Index":939,"Attempt":0,"Launch Time":1427397543653,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543661,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":410686,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2941,"Index":938,"Attempt":0,"Launch Time":1427397543652,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543661,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":438154,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2948,"Index":945,"Attempt":0,"Launch Time":1427397543663,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2943,"Index":940,"Attempt":0,"Launch Time":1427397543656,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543664,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":372093,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2949,"Index":946,"Attempt":0,"Launch Time":1427397543666,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2944,"Index":941,"Attempt":0,"Launch Time":1427397543657,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543666,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":457216,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2950,"Index":947,"Attempt":0,"Launch Time":1427397543668,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2947,"Index":944,"Attempt":0,"Launch Time":1427397543661,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543668,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":333981,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2951,"Index":948,"Attempt":0,"Launch Time":1427397543668,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2946,"Index":943,"Attempt":0,"Launch Time":1427397543660,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543668,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":304208,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2952,"Index":949,"Attempt":0,"Launch Time":1427397543682,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2949,"Index":946,"Attempt":0,"Launch Time":1427397543666,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543682,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":362618,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2953,"Index":950,"Attempt":0,"Launch Time":1427397543685,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2951,"Index":948,"Attempt":0,"Launch Time":1427397543668,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543685,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":16,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1444822,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2954,"Index":951,"Attempt":0,"Launch Time":1427397543691,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2952,"Index":949,"Attempt":0,"Launch Time":1427397543682,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543691,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":332714,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2955,"Index":952,"Attempt":0,"Launch Time":1427397543692,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2953,"Index":950,"Attempt":0,"Launch Time":1427397543685,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543692,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":332463,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2956,"Index":953,"Attempt":0,"Launch Time":1427397543694,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2903,"Index":900,"Attempt":0,"Launch Time":1427397543547,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543694,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":146,"Result Size":930,"JVM GC Time":19,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":373709,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2957,"Index":954,"Attempt":0,"Launch Time":1427397543701,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2955,"Index":952,"Attempt":0,"Launch Time":1427397543692,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543701,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":335402,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2958,"Index":955,"Attempt":0,"Launch Time":1427397543701,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2950,"Index":947,"Attempt":0,"Launch Time":1427397543668,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543701,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":33,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":333674,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2959,"Index":956,"Attempt":0,"Launch Time":1427397543702,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543712,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2954,"Index":951,"Attempt":0,"Launch Time":1427397543691,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543702,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":343865,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2960,"Index":957,"Attempt":0,"Launch Time":1427397543702,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543711,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2948,"Index":945,"Attempt":0,"Launch Time":1427397543663,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543702,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":33,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":365934,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2961,"Index":958,"Attempt":0,"Launch Time":1427397543703,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2912,"Index":909,"Attempt":0,"Launch Time":1427397543589,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543703,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":23,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":366516,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2962,"Index":959,"Attempt":0,"Launch Time":1427397543709,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2958,"Index":955,"Attempt":0,"Launch Time":1427397543701,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543709,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":335798,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2963,"Index":960,"Attempt":0,"Launch Time":1427397543711,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2960,"Index":957,"Attempt":0,"Launch Time":1427397543702,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543711,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":383398,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2964,"Index":961,"Attempt":0,"Launch Time":1427397543712,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2959,"Index":956,"Attempt":0,"Launch Time":1427397543702,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543712,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":358696,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2965,"Index":962,"Attempt":0,"Launch Time":1427397543715,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2956,"Index":953,"Attempt":0,"Launch Time":1427397543694,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543715,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":307806,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2966,"Index":963,"Attempt":0,"Launch Time":1427397543728,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2962,"Index":959,"Attempt":0,"Launch Time":1427397543709,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543728,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":325342,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2967,"Index":964,"Attempt":0,"Launch Time":1427397543729,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2963,"Index":960,"Attempt":0,"Launch Time":1427397543711,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543730,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":338335,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2911,"Index":908,"Attempt":0,"Launch Time":1427397543588,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543730,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":21,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":324219,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2968,"Index":965,"Attempt":0,"Launch Time":1427397543731,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2969,"Index":966,"Attempt":0,"Launch Time":1427397543731,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2965,"Index":962,"Attempt":0,"Launch Time":1427397543715,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543731,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":341296,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2970,"Index":967,"Attempt":0,"Launch Time":1427397543732,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2945,"Index":942,"Attempt":0,"Launch Time":1427397543658,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543732,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":32,"Executor Run Time":6,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":362655,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2961,"Index":958,"Attempt":0,"Launch Time":1427397543703,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543733,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":345413,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2971,"Index":968,"Attempt":0,"Launch Time":1427397543733,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2972,"Index":969,"Attempt":0,"Launch Time":1427397543739,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2967,"Index":964,"Attempt":0,"Launch Time":1427397543729,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543739,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":333258,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2973,"Index":970,"Attempt":0,"Launch Time":1427397543739,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2968,"Index":965,"Attempt":0,"Launch Time":1427397543731,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543740,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":340884,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2974,"Index":971,"Attempt":0,"Launch Time":1427397543740,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2969,"Index":966,"Attempt":0,"Launch Time":1427397543731,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543740,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":321916,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2975,"Index":972,"Attempt":0,"Launch Time":1427397543747,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2966,"Index":963,"Attempt":0,"Launch Time":1427397543728,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543747,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":294064,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2976,"Index":973,"Attempt":0,"Launch Time":1427397543748,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2972,"Index":969,"Attempt":0,"Launch Time":1427397543739,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543748,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":339999,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2977,"Index":974,"Attempt":0,"Launch Time":1427397543748,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2973,"Index":970,"Attempt":0,"Launch Time":1427397543739,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543749,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":317298,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2978,"Index":975,"Attempt":0,"Launch Time":1427397543754,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2977,"Index":974,"Attempt":0,"Launch Time":1427397543748,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543754,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":262907,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2979,"Index":976,"Attempt":0,"Launch Time":1427397543755,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2974,"Index":971,"Attempt":0,"Launch Time":1427397543740,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543756,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":393681,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2980,"Index":977,"Attempt":0,"Launch Time":1427397543758,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2976,"Index":973,"Attempt":0,"Launch Time":1427397543748,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543759,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":325967,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2981,"Index":978,"Attempt":0,"Launch Time":1427397543761,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2978,"Index":975,"Attempt":0,"Launch Time":1427397543754,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543761,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":340558,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2982,"Index":979,"Attempt":0,"Launch Time":1427397543763,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2979,"Index":976,"Attempt":0,"Launch Time":1427397543755,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543763,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":386648,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2983,"Index":980,"Attempt":0,"Launch Time":1427397543767,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2980,"Index":977,"Attempt":0,"Launch Time":1427397543758,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543767,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":371622,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2984,"Index":981,"Attempt":0,"Launch Time":1427397543768,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2981,"Index":978,"Attempt":0,"Launch Time":1427397543761,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543769,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":367220,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2985,"Index":982,"Attempt":0,"Launch Time":1427397543770,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2975,"Index":972,"Attempt":0,"Launch Time":1427397543747,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543770,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":367091,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2986,"Index":983,"Attempt":0,"Launch Time":1427397543771,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2982,"Index":979,"Attempt":0,"Launch Time":1427397543763,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543771,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":322269,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2987,"Index":984,"Attempt":0,"Launch Time":1427397543774,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2983,"Index":980,"Attempt":0,"Launch Time":1427397543767,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543775,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":373508,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2988,"Index":985,"Attempt":0,"Launch Time":1427397543776,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2984,"Index":981,"Attempt":0,"Launch Time":1427397543768,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543776,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":414503,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2989,"Index":986,"Attempt":0,"Launch Time":1427397543782,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2985,"Index":982,"Attempt":0,"Launch Time":1427397543770,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543782,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":438105,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2990,"Index":987,"Attempt":0,"Launch Time":1427397543783,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2987,"Index":984,"Attempt":0,"Launch Time":1427397543774,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543783,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":430686,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2991,"Index":988,"Attempt":0,"Launch Time":1427397543785,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2988,"Index":985,"Attempt":0,"Launch Time":1427397543776,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543785,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":473711,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2992,"Index":989,"Attempt":0,"Launch Time":1427397543785,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2986,"Index":983,"Attempt":0,"Launch Time":1427397543771,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543786,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":497203,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2993,"Index":990,"Attempt":0,"Launch Time":1427397543790,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2964,"Index":961,"Attempt":0,"Launch Time":1427397543712,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543790,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":77,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":34027291,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2994,"Index":991,"Attempt":0,"Launch Time":1427397543790,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2995,"Index":992,"Attempt":0,"Launch Time":1427397543790,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2989,"Index":986,"Attempt":0,"Launch Time":1427397543782,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543791,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":336099,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2990,"Index":987,"Attempt":0,"Launch Time":1427397543783,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543791,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":345766,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2996,"Index":993,"Attempt":0,"Launch Time":1427397543802,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2992,"Index":989,"Attempt":0,"Launch Time":1427397543785,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543802,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":330609,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2997,"Index":994,"Attempt":0,"Launch Time":1427397543808,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2996,"Index":993,"Attempt":0,"Launch Time":1427397543802,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543809,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":306839,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2998,"Index":995,"Attempt":0,"Launch Time":1427397543813,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2991,"Index":988,"Attempt":0,"Launch Time":1427397543785,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543813,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":27,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":491685,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2999,"Index":996,"Attempt":0,"Launch Time":1427397543814,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2993,"Index":990,"Attempt":0,"Launch Time":1427397543790,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543814,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":23,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":405241,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":3000,"Index":997,"Attempt":0,"Launch Time":1427397543816,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2994,"Index":991,"Attempt":0,"Launch Time":1427397543790,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543816,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":25,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":356505,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":3001,"Index":998,"Attempt":0,"Launch Time":1427397543817,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2995,"Index":992,"Attempt":0,"Launch Time":1427397543790,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543817,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":26,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":321239,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":3002,"Index":999,"Attempt":0,"Launch Time":1427397543818,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2997,"Index":994,"Attempt":0,"Launch Time":1427397543808,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543818,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":350128,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2998,"Index":995,"Attempt":0,"Launch Time":1427397543813,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543820,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":320246,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2999,"Index":996,"Attempt":0,"Launch Time":1427397543814,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543821,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":319459,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3000,"Index":997,"Attempt":0,"Launch Time":1427397543816,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543824,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":354077,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3001,"Index":998,"Attempt":0,"Launch Time":1427397543817,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543826,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":668882,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2970,"Index":967,"Attempt":0,"Launch Time":1427397543732,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543827,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":19,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":2935869,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3002,"Index":999,"Attempt":0,"Launch Time":1427397543818,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543828,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":402588,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2957,"Index":954,"Attempt":0,"Launch Time":1427397543701,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543831,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":120,"Executor Run Time":9,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":449183,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2971,"Index":968,"Attempt":0,"Launch Time":1427397543733,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543831,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":26,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":441916,"Shuffle Records Written":100}}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":6,"Stage Attempt ID":0,"Stage Name":"map at :21","Number of Tasks":1000,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:287)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:21)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:28)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:30)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.(:32)\n$line15.$read$$iwC$$iwC$$iwC.(:34)\n$line15.$read$$iwC$$iwC.(:36)\n$line15.$read$$iwC.(:38)\n$line15.$read.(:40)\n$line15.$read$.(:44)\n$line15.$read$.()\n$line15.$eval$.(:7)\n$line15.$eval$.()\n$line15.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Submission Time":1427397541495,"Completion Time":1427397543832,"Accumulables":[]}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":7,"Stage Attempt ID":0,"Stage Name":"next at :26","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":50,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"scala.collection.Iterator$$anon$13.next(Iterator.scala:372)\n$line26.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line26.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:31)\n$line26.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:33)\n$line26.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:35)\n$line26.$read$$iwC$$iwC$$iwC$$iwC.(:37)\n$line26.$read$$iwC$$iwC$$iwC.(:39)\n$line26.$read$$iwC$$iwC.(:41)\n$line26.$read$$iwC.(:43)\n$line26.$read.(:45)\n$line26.$read$.(:49)\n$line26.$read$.()\n$line26.$eval$.(:7)\n$line26.$eval$.()\n$line26.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":7,"Stage Attempt ID":0,"Task Info":{"Task ID":3003,"Index":0,"Attempt":0,"Launch Time":1427397543837,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":7,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3003,"Index":0,"Attempt":0,"Launch Time":1427397543837,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543951,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":113,"Result Size":1060,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":1000,"Fetch Wait Time":1,"Remote Bytes Read":0,"Local Bytes Read":192030,"Total Records Read":2000}}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":7,"Stage Attempt ID":0,"Stage Name":"next at :26","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":50,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"scala.collection.Iterator$$anon$13.next(Iterator.scala:372)\n$line26.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line26.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:31)\n$line26.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:33)\n$line26.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:35)\n$line26.$read$$iwC$$iwC$$iwC$$iwC.(:37)\n$line26.$read$$iwC$$iwC$$iwC.(:39)\n$line26.$read$$iwC$$iwC.(:41)\n$line26.$read$$iwC.(:43)\n$line26.$read.(:45)\n$line26.$read$.(:49)\n$line26.$read$.()\n$line26.$eval$.(:7)\n$line26.$eval$.()\n$line26.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Submission Time":1427397543837,"Completion Time":1427397543951,"Accumulables":[]}} -{"Event":"SparkListenerJobEnd","Job ID":3,"Completion Time":1427397543952,"Job Result":{"Result":"JobSucceeded"}} -{"Event":"SparkListenerJobStart","Job ID":4,"Submission Time":1427397546619,"Stage Infos":[{"Stage ID":9,"Stage Attempt ID":0,"Stage Name":"next at :26","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":50,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"scala.collection.Iterator$$anon$13.next(Iterator.scala:372)\n$line28.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line28.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:31)\n$line28.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:33)\n$line28.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:35)\n$line28.$read$$iwC$$iwC$$iwC$$iwC.(:37)\n$line28.$read$$iwC$$iwC$$iwC.(:39)\n$line28.$read$$iwC$$iwC.(:41)\n$line28.$read$$iwC.(:43)\n$line28.$read.(:45)\n$line28.$read$.(:49)\n$line28.$read$.()\n$line28.$eval$.(:7)\n$line28.$eval$.()\n$line28.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]},{"Stage ID":8,"Stage Attempt ID":0,"Stage Name":"map at :21","Number of Tasks":1000,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:287)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:21)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:28)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:30)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.(:32)\n$line15.$read$$iwC$$iwC$$iwC.(:34)\n$line15.$read$$iwC$$iwC.(:36)\n$line15.$read$$iwC.(:38)\n$line15.$read.(:40)\n$line15.$read$.(:44)\n$line15.$read$.()\n$line15.$eval$.(:7)\n$line15.$eval$.()\n$line15.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]}],"Stage IDs":[9,8]} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":9,"Stage Attempt ID":0,"Stage Name":"next at :26","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":50,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"scala.collection.Iterator$$anon$13.next(Iterator.scala:372)\n$line28.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line28.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:31)\n$line28.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:33)\n$line28.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:35)\n$line28.$read$$iwC$$iwC$$iwC$$iwC.(:37)\n$line28.$read$$iwC$$iwC$$iwC.(:39)\n$line28.$read$$iwC$$iwC.(:41)\n$line28.$read$$iwC.(:43)\n$line28.$read.(:45)\n$line28.$read$.(:49)\n$line28.$read$.()\n$line28.$eval$.(:7)\n$line28.$eval$.()\n$line28.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":9,"Stage Attempt ID":0,"Task Info":{"Task ID":3004,"Index":0,"Attempt":0,"Launch Time":1427397546623,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":9,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3004,"Index":0,"Attempt":0,"Launch Time":1427397546623,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397546709,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":86,"Result Size":1060,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":1000,"Fetch Wait Time":1,"Remote Bytes Read":0,"Local Bytes Read":192019,"Total Records Read":2000}}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":9,"Stage Attempt ID":0,"Stage Name":"next at :26","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":50,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"scala.collection.Iterator$$anon$13.next(Iterator.scala:372)\n$line28.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line28.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:31)\n$line28.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:33)\n$line28.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:35)\n$line28.$read$$iwC$$iwC$$iwC$$iwC.(:37)\n$line28.$read$$iwC$$iwC$$iwC.(:39)\n$line28.$read$$iwC$$iwC.(:41)\n$line28.$read$$iwC.(:43)\n$line28.$read.(:45)\n$line28.$read$.(:49)\n$line28.$read$.()\n$line28.$eval$.(:7)\n$line28.$eval$.()\n$line28.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Submission Time":1427397546623,"Completion Time":1427397546710,"Accumulables":[]}} -{"Event":"SparkListenerJobEnd","Job ID":4,"Completion Time":1427397546710,"Job Result":{"Result":"JobSucceeded"}} -{"Event":"SparkListenerJobStart","Job ID":5,"Submission Time":1427397548267,"Stage Infos":[{"Stage ID":10,"Stage Attempt ID":0,"Stage Name":"map at :21","Number of Tasks":1000,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:287)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:21)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:28)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:30)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.(:32)\n$line15.$read$$iwC$$iwC$$iwC.(:34)\n$line15.$read$$iwC$$iwC.(:36)\n$line15.$read$$iwC.(:38)\n$line15.$read.(:40)\n$line15.$read$.(:44)\n$line15.$read$.()\n$line15.$eval$.(:7)\n$line15.$eval$.()\n$line15.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]},{"Stage ID":11,"Stage Attempt ID":0,"Stage Name":"next at :26","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":50,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"scala.collection.Iterator$$anon$13.next(Iterator.scala:372)\n$line30.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line30.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:31)\n$line30.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:33)\n$line30.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:35)\n$line30.$read$$iwC$$iwC$$iwC$$iwC.(:37)\n$line30.$read$$iwC$$iwC$$iwC.(:39)\n$line30.$read$$iwC$$iwC.(:41)\n$line30.$read$$iwC.(:43)\n$line30.$read.(:45)\n$line30.$read$.(:49)\n$line30.$read$.()\n$line30.$eval$.(:7)\n$line30.$eval$.()\n$line30.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]}],"Stage IDs":[10,11]} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":11,"Stage Attempt ID":0,"Stage Name":"next at :26","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":50,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"scala.collection.Iterator$$anon$13.next(Iterator.scala:372)\n$line30.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line30.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:31)\n$line30.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:33)\n$line30.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:35)\n$line30.$read$$iwC$$iwC$$iwC$$iwC.(:37)\n$line30.$read$$iwC$$iwC$$iwC.(:39)\n$line30.$read$$iwC$$iwC.(:41)\n$line30.$read$$iwC.(:43)\n$line30.$read.(:45)\n$line30.$read$.(:49)\n$line30.$read$.()\n$line30.$eval$.(:7)\n$line30.$eval$.()\n$line30.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":11,"Stage Attempt ID":0,"Task Info":{"Task ID":3005,"Index":0,"Attempt":0,"Launch Time":1427397548270,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":11,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3005,"Index":0,"Attempt":0,"Launch Time":1427397548270,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397548375,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":104,"Result Size":1060,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":1000,"Fetch Wait Time":1,"Remote Bytes Read":0,"Local Bytes Read":192019,"Total Records Read":2000}}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":11,"Stage Attempt ID":0,"Stage Name":"next at :26","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":50,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"scala.collection.Iterator$$anon$13.next(Iterator.scala:372)\n$line30.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line30.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:31)\n$line30.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:33)\n$line30.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:35)\n$line30.$read$$iwC$$iwC$$iwC$$iwC.(:37)\n$line30.$read$$iwC$$iwC$$iwC.(:39)\n$line30.$read$$iwC$$iwC.(:41)\n$line30.$read$$iwC.(:43)\n$line30.$read.(:45)\n$line30.$read$.(:49)\n$line30.$read$.()\n$line30.$eval$.(:7)\n$line30.$eval$.()\n$line30.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Submission Time":1427397548270,"Completion Time":1427397548375,"Accumulables":[]}} -{"Event":"SparkListenerJobEnd","Job ID":5,"Completion Time":1427397548375,"Job Result":{"Result":"JobSucceeded"}} -{"Event":"SparkListenerJobStart","Job ID":6,"Submission Time":1427397549487,"Stage Infos":[{"Stage ID":12,"Stage Attempt ID":0,"Stage Name":"map at :21","Number of Tasks":1000,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:287)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:21)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:28)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:30)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.(:32)\n$line15.$read$$iwC$$iwC$$iwC.(:34)\n$line15.$read$$iwC$$iwC.(:36)\n$line15.$read$$iwC.(:38)\n$line15.$read.(:40)\n$line15.$read$.(:44)\n$line15.$read$.()\n$line15.$eval$.(:7)\n$line15.$eval$.()\n$line15.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]},{"Stage ID":13,"Stage Attempt ID":0,"Stage Name":"next at :26","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":50,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"scala.collection.Iterator$$anon$13.next(Iterator.scala:372)\n$line32.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line32.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:31)\n$line32.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:33)\n$line32.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:35)\n$line32.$read$$iwC$$iwC$$iwC$$iwC.(:37)\n$line32.$read$$iwC$$iwC$$iwC.(:39)\n$line32.$read$$iwC$$iwC.(:41)\n$line32.$read$$iwC.(:43)\n$line32.$read.(:45)\n$line32.$read$.(:49)\n$line32.$read$.()\n$line32.$eval$.(:7)\n$line32.$eval$.()\n$line32.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]}],"Stage IDs":[12,13]} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":12,"Stage Attempt ID":0,"Stage Name":"map at :21","Number of Tasks":1000,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:287)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:21)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:28)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:30)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.(:32)\n$line15.$read$$iwC$$iwC$$iwC.(:34)\n$line15.$read$$iwC$$iwC.(:36)\n$line15.$read$$iwC.(:38)\n$line15.$read.(:40)\n$line15.$read$.(:44)\n$line15.$read$.()\n$line15.$eval$.(:7)\n$line15.$eval$.()\n$line15.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3006,"Index":0,"Attempt":0,"Launch Time":1427397549496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3007,"Index":1,"Attempt":0,"Launch Time":1427397549496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3008,"Index":2,"Attempt":0,"Launch Time":1427397549496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3009,"Index":3,"Attempt":0,"Launch Time":1427397549496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3010,"Index":4,"Attempt":0,"Launch Time":1427397549496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3011,"Index":5,"Attempt":0,"Launch Time":1427397549496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3012,"Index":6,"Attempt":0,"Launch Time":1427397549496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3013,"Index":7,"Attempt":0,"Launch Time":1427397549496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3014,"Index":8,"Attempt":0,"Launch Time":1427397549504,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3011,"Index":5,"Attempt":0,"Launch Time":1427397549496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549514,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":303786,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3008,"Index":2,"Attempt":0,"Launch Time":1427397549496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549515,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":302702,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3015,"Index":9,"Attempt":0,"Launch Time":1427397549515,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3016,"Index":10,"Attempt":0,"Launch Time":1427397549515,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3017,"Index":11,"Attempt":0,"Launch Time":1427397549515,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3009,"Index":3,"Attempt":0,"Launch Time":1427397549496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549515,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":276274,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3006,"Index":0,"Attempt":0,"Launch Time":1427397549496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549516,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":297081,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3018,"Index":12,"Attempt":0,"Launch Time":1427397549517,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3019,"Index":13,"Attempt":0,"Launch Time":1427397549517,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3010,"Index":4,"Attempt":0,"Launch Time":1427397549496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549517,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":19,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":301871,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3012,"Index":6,"Attempt":0,"Launch Time":1427397549496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549517,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":345554,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3020,"Index":14,"Attempt":0,"Launch Time":1427397549520,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3007,"Index":1,"Attempt":0,"Launch Time":1427397549496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549522,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":24,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":305324,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3021,"Index":15,"Attempt":0,"Launch Time":1427397549524,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3022,"Index":16,"Attempt":0,"Launch Time":1427397549524,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3023,"Index":17,"Attempt":0,"Launch Time":1427397549526,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3018,"Index":12,"Attempt":0,"Launch Time":1427397549517,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549526,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":301855,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3016,"Index":10,"Attempt":0,"Launch Time":1427397549515,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549526,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":374792,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3024,"Index":18,"Attempt":0,"Launch Time":1427397549526,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3025,"Index":19,"Attempt":0,"Launch Time":1427397549526,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3013,"Index":7,"Attempt":0,"Launch Time":1427397549496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549527,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":25,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1447021,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3015,"Index":9,"Attempt":0,"Launch Time":1427397549515,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549527,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":633706,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3014,"Index":8,"Attempt":0,"Launch Time":1427397549504,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549528,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":303249,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3026,"Index":20,"Attempt":0,"Launch Time":1427397549528,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3027,"Index":21,"Attempt":0,"Launch Time":1427397549529,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3019,"Index":13,"Attempt":0,"Launch Time":1427397549517,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549530,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":307508,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3028,"Index":22,"Attempt":0,"Launch Time":1427397549534,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3029,"Index":23,"Attempt":0,"Launch Time":1427397549534,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3020,"Index":14,"Attempt":0,"Launch Time":1427397549520,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549534,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":266687,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3022,"Index":16,"Attempt":0,"Launch Time":1427397549524,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549535,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":326511,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3017,"Index":11,"Attempt":0,"Launch Time":1427397549515,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549536,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1418988,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3030,"Index":24,"Attempt":0,"Launch Time":1427397549536,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3031,"Index":25,"Attempt":0,"Launch Time":1427397549536,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3032,"Index":26,"Attempt":0,"Launch Time":1427397549536,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3033,"Index":27,"Attempt":0,"Launch Time":1427397549537,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3034,"Index":28,"Attempt":0,"Launch Time":1427397549538,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3035,"Index":29,"Attempt":0,"Launch Time":1427397549538,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3025,"Index":19,"Attempt":0,"Launch Time":1427397549526,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549538,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":357151,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3023,"Index":17,"Attempt":0,"Launch Time":1427397549526,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549538,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":312482,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3026,"Index":20,"Attempt":0,"Launch Time":1427397549528,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549538,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":288507,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3021,"Index":15,"Attempt":0,"Launch Time":1427397549524,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549539,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":326274,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3024,"Index":18,"Attempt":0,"Launch Time":1427397549526,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549545,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":387862,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3036,"Index":30,"Attempt":0,"Launch Time":1427397549545,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3037,"Index":31,"Attempt":0,"Launch Time":1427397549546,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3028,"Index":22,"Attempt":0,"Launch Time":1427397549534,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549546,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":274652,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3029,"Index":23,"Attempt":0,"Launch Time":1427397549534,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549546,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":368998,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3027,"Index":21,"Attempt":0,"Launch Time":1427397549529,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549547,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":288479,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3038,"Index":32,"Attempt":0,"Launch Time":1427397549547,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3039,"Index":33,"Attempt":0,"Launch Time":1427397549548,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3040,"Index":34,"Attempt":0,"Launch Time":1427397549548,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3041,"Index":35,"Attempt":0,"Launch Time":1427397549548,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3042,"Index":36,"Attempt":0,"Launch Time":1427397549548,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3032,"Index":26,"Attempt":0,"Launch Time":1427397549536,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549548,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":398608,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3034,"Index":28,"Attempt":0,"Launch Time":1427397549538,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549550,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":405084,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3043,"Index":37,"Attempt":0,"Launch Time":1427397549552,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3044,"Index":38,"Attempt":0,"Launch Time":1427397549552,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3033,"Index":27,"Attempt":0,"Launch Time":1427397549537,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549552,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":411363,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3035,"Index":29,"Attempt":0,"Launch Time":1427397549538,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549552,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":364295,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3036,"Index":30,"Attempt":0,"Launch Time":1427397549545,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549552,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":318140,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3031,"Index":25,"Attempt":0,"Launch Time":1427397549536,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549556,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":431403,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3045,"Index":39,"Attempt":0,"Launch Time":1427397549557,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3046,"Index":40,"Attempt":0,"Launch Time":1427397549558,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3047,"Index":41,"Attempt":0,"Launch Time":1427397549558,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3048,"Index":42,"Attempt":0,"Launch Time":1427397549559,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3049,"Index":43,"Attempt":0,"Launch Time":1427397549559,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3050,"Index":44,"Attempt":0,"Launch Time":1427397549562,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3051,"Index":45,"Attempt":0,"Launch Time":1427397549562,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3052,"Index":46,"Attempt":0,"Launch Time":1427397549562,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3030,"Index":24,"Attempt":0,"Launch Time":1427397549536,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549563,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":462571,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3042,"Index":36,"Attempt":0,"Launch Time":1427397549548,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549563,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":307346,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3041,"Index":35,"Attempt":0,"Launch Time":1427397549548,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549563,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":297041,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3039,"Index":33,"Attempt":0,"Launch Time":1427397549548,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549563,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":309493,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3043,"Index":37,"Attempt":0,"Launch Time":1427397549552,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549563,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":304146,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3044,"Index":38,"Attempt":0,"Launch Time":1427397549552,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549563,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":303019,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3040,"Index":34,"Attempt":0,"Launch Time":1427397549548,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549564,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":299037,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3038,"Index":32,"Attempt":0,"Launch Time":1427397549547,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549567,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":284406,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3037,"Index":31,"Attempt":0,"Launch Time":1427397549546,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549567,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":271847,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3053,"Index":47,"Attempt":0,"Launch Time":1427397549568,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3054,"Index":48,"Attempt":0,"Launch Time":1427397549568,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3047,"Index":41,"Attempt":0,"Launch Time":1427397549558,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549568,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":340922,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3045,"Index":39,"Attempt":0,"Launch Time":1427397549557,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549568,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":325471,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3055,"Index":49,"Attempt":0,"Launch Time":1427397549568,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3048,"Index":42,"Attempt":0,"Launch Time":1427397549559,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549569,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":332855,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3056,"Index":50,"Attempt":0,"Launch Time":1427397549573,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3057,"Index":51,"Attempt":0,"Launch Time":1427397549573,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3058,"Index":52,"Attempt":0,"Launch Time":1427397549575,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3059,"Index":53,"Attempt":0,"Launch Time":1427397549575,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3046,"Index":40,"Attempt":0,"Launch Time":1427397549558,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549575,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":294717,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3049,"Index":43,"Attempt":0,"Launch Time":1427397549559,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549575,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":303506,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3050,"Index":44,"Attempt":0,"Launch Time":1427397549562,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549576,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":317180,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3052,"Index":46,"Attempt":0,"Launch Time":1427397549562,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549576,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":312931,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3060,"Index":54,"Attempt":0,"Launch Time":1427397549577,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3061,"Index":55,"Attempt":0,"Launch Time":1427397549577,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3062,"Index":56,"Attempt":0,"Launch Time":1427397549577,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3063,"Index":57,"Attempt":0,"Launch Time":1427397549577,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3051,"Index":45,"Attempt":0,"Launch Time":1427397549562,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549577,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":5,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":307591,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3053,"Index":47,"Attempt":0,"Launch Time":1427397549568,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549578,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":302252,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3055,"Index":49,"Attempt":0,"Launch Time":1427397549568,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549578,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":320592,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3054,"Index":48,"Attempt":0,"Launch Time":1427397549568,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549578,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":300711,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3064,"Index":58,"Attempt":0,"Launch Time":1427397549601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3065,"Index":59,"Attempt":0,"Launch Time":1427397549602,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3066,"Index":60,"Attempt":0,"Launch Time":1427397549603,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3067,"Index":61,"Attempt":0,"Launch Time":1427397549605,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3068,"Index":62,"Attempt":0,"Launch Time":1427397549606,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3069,"Index":63,"Attempt":0,"Launch Time":1427397549612,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3070,"Index":64,"Attempt":0,"Launch Time":1427397549614,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3071,"Index":65,"Attempt":0,"Launch Time":1427397549615,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3072,"Index":66,"Attempt":0,"Launch Time":1427397549621,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3073,"Index":67,"Attempt":0,"Launch Time":1427397549622,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3074,"Index":68,"Attempt":0,"Launch Time":1427397549627,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3075,"Index":69,"Attempt":0,"Launch Time":1427397549628,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3076,"Index":70,"Attempt":0,"Launch Time":1427397549630,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3077,"Index":71,"Attempt":0,"Launch Time":1427397549634,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3078,"Index":72,"Attempt":0,"Launch Time":1427397549634,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3079,"Index":73,"Attempt":0,"Launch Time":1427397549638,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3080,"Index":74,"Attempt":0,"Launch Time":1427397549639,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3057,"Index":51,"Attempt":0,"Launch Time":1427397549573,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549639,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":26,"Result Size":930,"JVM GC Time":18,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":370473,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3061,"Index":55,"Attempt":0,"Launch Time":1427397549577,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549640,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":24,"Result Size":930,"JVM GC Time":18,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":306237,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3058,"Index":52,"Attempt":0,"Launch Time":1427397549575,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549640,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":26,"Result Size":930,"JVM GC Time":18,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":321793,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3062,"Index":56,"Attempt":0,"Launch Time":1427397549577,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549640,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":21,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":299987,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3067,"Index":61,"Attempt":0,"Launch Time":1427397549605,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549640,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":334124,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3070,"Index":64,"Attempt":0,"Launch Time":1427397549614,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549640,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":316217,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3063,"Index":57,"Attempt":0,"Launch Time":1427397549577,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549640,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":21,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":266148,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3069,"Index":63,"Attempt":0,"Launch Time":1427397549612,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549640,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":446110,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3065,"Index":59,"Attempt":0,"Launch Time":1427397549602,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549641,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":20,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9610,"Shuffle Write Time":2040919,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3071,"Index":65,"Attempt":0,"Launch Time":1427397549615,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549641,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":360161,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3068,"Index":62,"Attempt":0,"Launch Time":1427397549606,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549641,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":317833,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3073,"Index":67,"Attempt":0,"Launch Time":1427397549622,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549641,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9614,"Shuffle Write Time":396314,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3066,"Index":60,"Attempt":0,"Launch Time":1427397549603,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549641,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":342871,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3076,"Index":70,"Attempt":0,"Launch Time":1427397549630,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549641,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":329537,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3074,"Index":68,"Attempt":0,"Launch Time":1427397549627,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549641,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":332201,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3081,"Index":75,"Attempt":0,"Launch Time":1427397549642,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3082,"Index":76,"Attempt":0,"Launch Time":1427397549642,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3072,"Index":66,"Attempt":0,"Launch Time":1427397549621,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549642,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":441841,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3077,"Index":71,"Attempt":0,"Launch Time":1427397549634,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549642,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":302179,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3064,"Index":58,"Attempt":0,"Launch Time":1427397549601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549643,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":37,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":461624,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3075,"Index":69,"Attempt":0,"Launch Time":1427397549628,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549643,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":316157,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3083,"Index":77,"Attempt":0,"Launch Time":1427397549647,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3079,"Index":73,"Attempt":0,"Launch Time":1427397549638,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549647,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":394675,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3084,"Index":78,"Attempt":0,"Launch Time":1427397549649,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3059,"Index":53,"Attempt":0,"Launch Time":1427397549575,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549649,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":72,"Result Size":930,"JVM GC Time":18,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":474463,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3085,"Index":79,"Attempt":0,"Launch Time":1427397549649,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3060,"Index":54,"Attempt":0,"Launch Time":1427397549577,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549650,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":72,"Result Size":930,"JVM GC Time":18,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":328460,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3086,"Index":80,"Attempt":0,"Launch Time":1427397549651,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3087,"Index":81,"Attempt":0,"Launch Time":1427397549652,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3081,"Index":75,"Attempt":0,"Launch Time":1427397549642,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549652,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":315548,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3082,"Index":76,"Attempt":0,"Launch Time":1427397549642,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549652,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":306893,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3088,"Index":82,"Attempt":0,"Launch Time":1427397549652,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3078,"Index":72,"Attempt":0,"Launch Time":1427397549634,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549652,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":4972058,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3089,"Index":83,"Attempt":0,"Launch Time":1427397549653,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3056,"Index":50,"Attempt":0,"Launch Time":1427397549573,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549653,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":77,"Result Size":930,"JVM GC Time":18,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":344585,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3090,"Index":84,"Attempt":0,"Launch Time":1427397549655,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3083,"Index":77,"Attempt":0,"Launch Time":1427397549647,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549655,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":282187,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3091,"Index":85,"Attempt":0,"Launch Time":1427397549656,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3080,"Index":74,"Attempt":0,"Launch Time":1427397549639,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549656,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":628401,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3092,"Index":86,"Attempt":0,"Launch Time":1427397549659,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3088,"Index":82,"Attempt":0,"Launch Time":1427397549652,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549659,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":309171,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3093,"Index":87,"Attempt":0,"Launch Time":1427397549661,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3089,"Index":83,"Attempt":0,"Launch Time":1427397549653,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549661,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":321286,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3094,"Index":88,"Attempt":0,"Launch Time":1427397549662,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3090,"Index":84,"Attempt":0,"Launch Time":1427397549655,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549662,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":318687,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3095,"Index":89,"Attempt":0,"Launch Time":1427397549664,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3086,"Index":80,"Attempt":0,"Launch Time":1427397549651,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549664,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":388862,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3096,"Index":90,"Attempt":0,"Launch Time":1427397549666,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549673,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3091,"Index":85,"Attempt":0,"Launch Time":1427397549656,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549666,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":313649,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3097,"Index":91,"Attempt":0,"Launch Time":1427397549666,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549699,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3092,"Index":86,"Attempt":0,"Launch Time":1427397549659,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549667,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":277912,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3098,"Index":92,"Attempt":0,"Launch Time":1427397549669,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549677,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3093,"Index":87,"Attempt":0,"Launch Time":1427397549661,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549669,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":297995,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3099,"Index":93,"Attempt":0,"Launch Time":1427397549670,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549678,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3094,"Index":88,"Attempt":0,"Launch Time":1427397549662,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549670,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":273382,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3100,"Index":94,"Attempt":0,"Launch Time":1427397549672,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549680,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3095,"Index":89,"Attempt":0,"Launch Time":1427397549664,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549672,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":314789,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3101,"Index":95,"Attempt":0,"Launch Time":1427397549672,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549680,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3096,"Index":90,"Attempt":0,"Launch Time":1427397549666,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549673,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":309898,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3102,"Index":96,"Attempt":0,"Launch Time":1427397549677,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549699,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3098,"Index":92,"Attempt":0,"Launch Time":1427397549669,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549677,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":309600,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3103,"Index":97,"Attempt":0,"Launch Time":1427397549678,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549696,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3099,"Index":93,"Attempt":0,"Launch Time":1427397549670,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549678,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":363292,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3104,"Index":98,"Attempt":0,"Launch Time":1427397549679,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549705,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3085,"Index":79,"Attempt":0,"Launch Time":1427397549649,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549679,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":28,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":424375,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3105,"Index":99,"Attempt":0,"Launch Time":1427397549680,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549695,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3106,"Index":100,"Attempt":0,"Launch Time":1427397549680,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549725,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3100,"Index":94,"Attempt":0,"Launch Time":1427397549672,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549680,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":364656,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3101,"Index":95,"Attempt":0,"Launch Time":1427397549672,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549680,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":371110,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3107,"Index":101,"Attempt":0,"Launch Time":1427397549693,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3087,"Index":81,"Attempt":0,"Launch Time":1427397549652,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549693,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":40,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":318687,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3108,"Index":102,"Attempt":0,"Launch Time":1427397549695,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549703,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3105,"Index":99,"Attempt":0,"Launch Time":1427397549680,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549695,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":356227,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3109,"Index":103,"Attempt":0,"Launch Time":1427397549695,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549703,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3103,"Index":97,"Attempt":0,"Launch Time":1427397549678,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549696,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":402468,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3110,"Index":104,"Attempt":0,"Launch Time":1427397549698,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549707,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3111,"Index":105,"Attempt":0,"Launch Time":1427397549699,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549707,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3084,"Index":78,"Attempt":0,"Launch Time":1427397549649,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549699,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":49,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":330014,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3097,"Index":91,"Attempt":0,"Launch Time":1427397549666,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549699,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":22,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":365564,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3112,"Index":106,"Attempt":0,"Launch Time":1427397549699,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549706,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3102,"Index":96,"Attempt":0,"Launch Time":1427397549677,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549699,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":20,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":348803,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3113,"Index":107,"Attempt":0,"Launch Time":1427397549702,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549709,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3109,"Index":103,"Attempt":0,"Launch Time":1427397549695,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549703,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":313592,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3114,"Index":108,"Attempt":0,"Launch Time":1427397549703,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549710,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3108,"Index":102,"Attempt":0,"Launch Time":1427397549695,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549703,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":298696,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3115,"Index":109,"Attempt":0,"Launch Time":1427397549705,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549714,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3104,"Index":98,"Attempt":0,"Launch Time":1427397549679,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549705,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":24,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":2719222,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3112,"Index":106,"Attempt":0,"Launch Time":1427397549699,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549706,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":323473,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3116,"Index":110,"Attempt":0,"Launch Time":1427397549706,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549713,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3117,"Index":111,"Attempt":0,"Launch Time":1427397549707,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549728,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3111,"Index":105,"Attempt":0,"Launch Time":1427397549699,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549707,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":317646,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3118,"Index":112,"Attempt":0,"Launch Time":1427397549707,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3110,"Index":104,"Attempt":0,"Launch Time":1427397549698,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549707,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":272035,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3119,"Index":113,"Attempt":0,"Launch Time":1427397549709,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549716,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3113,"Index":107,"Attempt":0,"Launch Time":1427397549702,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549709,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":354368,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3120,"Index":114,"Attempt":0,"Launch Time":1427397549710,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549725,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3114,"Index":108,"Attempt":0,"Launch Time":1427397549703,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549710,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":341950,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3121,"Index":115,"Attempt":0,"Launch Time":1427397549713,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549726,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3116,"Index":110,"Attempt":0,"Launch Time":1427397549706,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549713,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":267819,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3122,"Index":116,"Attempt":0,"Launch Time":1427397549713,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549720,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3115,"Index":109,"Attempt":0,"Launch Time":1427397549705,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549714,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":400922,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3123,"Index":117,"Attempt":0,"Launch Time":1427397549716,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549723,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3119,"Index":113,"Attempt":0,"Launch Time":1427397549709,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549716,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":272712,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3124,"Index":118,"Attempt":0,"Launch Time":1427397549720,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3122,"Index":116,"Attempt":0,"Launch Time":1427397549713,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549720,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":272628,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3125,"Index":119,"Attempt":0,"Launch Time":1427397549723,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3123,"Index":117,"Attempt":0,"Launch Time":1427397549716,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549723,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":307550,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3126,"Index":120,"Attempt":0,"Launch Time":1427397549725,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549732,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3106,"Index":100,"Attempt":0,"Launch Time":1427397549680,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549725,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":42,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":336601,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3127,"Index":121,"Attempt":0,"Launch Time":1427397549725,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3120,"Index":114,"Attempt":0,"Launch Time":1427397549710,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549725,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":304929,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3128,"Index":122,"Attempt":0,"Launch Time":1427397549726,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3121,"Index":115,"Attempt":0,"Launch Time":1427397549713,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549726,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":5256232,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3129,"Index":123,"Attempt":0,"Launch Time":1427397549728,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3117,"Index":111,"Attempt":0,"Launch Time":1427397549707,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549728,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":941933,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3130,"Index":124,"Attempt":0,"Launch Time":1427397549732,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3126,"Index":120,"Attempt":0,"Launch Time":1427397549725,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549732,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":318688,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3131,"Index":125,"Attempt":0,"Launch Time":1427397549737,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3118,"Index":112,"Attempt":0,"Launch Time":1427397549707,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549737,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":286535,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3132,"Index":126,"Attempt":0,"Launch Time":1427397549737,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3127,"Index":121,"Attempt":0,"Launch Time":1427397549725,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549737,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":313749,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3133,"Index":127,"Attempt":0,"Launch Time":1427397549737,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3129,"Index":123,"Attempt":0,"Launch Time":1427397549728,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549738,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":325048,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3134,"Index":128,"Attempt":0,"Launch Time":1427397549738,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3125,"Index":119,"Attempt":0,"Launch Time":1427397549723,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549738,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":331932,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3135,"Index":129,"Attempt":0,"Launch Time":1427397549739,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3107,"Index":101,"Attempt":0,"Launch Time":1427397549693,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549739,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":35,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":447854,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3136,"Index":130,"Attempt":0,"Launch Time":1427397549745,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3132,"Index":126,"Attempt":0,"Launch Time":1427397549737,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549746,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":346323,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3137,"Index":131,"Attempt":0,"Launch Time":1427397549748,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3135,"Index":129,"Attempt":0,"Launch Time":1427397549739,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549748,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":366927,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3138,"Index":132,"Attempt":0,"Launch Time":1427397549748,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3133,"Index":127,"Attempt":0,"Launch Time":1427397549737,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549748,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":365174,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3139,"Index":133,"Attempt":0,"Launch Time":1427397549753,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3136,"Index":130,"Attempt":0,"Launch Time":1427397549745,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549753,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":311008,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3140,"Index":134,"Attempt":0,"Launch Time":1427397549754,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3137,"Index":131,"Attempt":0,"Launch Time":1427397549748,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549755,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":310122,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3141,"Index":135,"Attempt":0,"Launch Time":1427397549755,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3138,"Index":132,"Attempt":0,"Launch Time":1427397549748,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549756,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":339457,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3142,"Index":136,"Attempt":0,"Launch Time":1427397549763,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3139,"Index":133,"Attempt":0,"Launch Time":1427397549753,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549763,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":318919,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3143,"Index":137,"Attempt":0,"Launch Time":1427397549763,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3140,"Index":134,"Attempt":0,"Launch Time":1427397549754,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549763,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":360845,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3144,"Index":138,"Attempt":0,"Launch Time":1427397549763,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3141,"Index":135,"Attempt":0,"Launch Time":1427397549755,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549764,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":329998,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3145,"Index":139,"Attempt":0,"Launch Time":1427397549765,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3131,"Index":125,"Attempt":0,"Launch Time":1427397549737,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549765,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":272842,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3146,"Index":140,"Attempt":0,"Launch Time":1427397549771,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3144,"Index":138,"Attempt":0,"Launch Time":1427397549763,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549771,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":428495,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3147,"Index":141,"Attempt":0,"Launch Time":1427397549772,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3143,"Index":137,"Attempt":0,"Launch Time":1427397549763,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549772,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":388162,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3148,"Index":142,"Attempt":0,"Launch Time":1427397549773,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3142,"Index":136,"Attempt":0,"Launch Time":1427397549763,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549773,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":368421,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3149,"Index":143,"Attempt":0,"Launch Time":1427397549774,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3145,"Index":139,"Attempt":0,"Launch Time":1427397549765,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549774,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":365908,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3150,"Index":144,"Attempt":0,"Launch Time":1427397549778,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3128,"Index":122,"Attempt":0,"Launch Time":1427397549726,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549778,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":51,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":333164,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3151,"Index":145,"Attempt":0,"Launch Time":1427397549786,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3146,"Index":140,"Attempt":0,"Launch Time":1427397549771,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549787,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":14,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9606,"Shuffle Write Time":531674,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3152,"Index":146,"Attempt":0,"Launch Time":1427397549788,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3148,"Index":142,"Attempt":0,"Launch Time":1427397549773,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549788,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":355144,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3153,"Index":147,"Attempt":0,"Launch Time":1427397549794,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3151,"Index":145,"Attempt":0,"Launch Time":1427397549786,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549794,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":324859,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3152,"Index":146,"Attempt":0,"Launch Time":1427397549788,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549795,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":332480,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3154,"Index":148,"Attempt":0,"Launch Time":1427397549795,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3155,"Index":149,"Attempt":0,"Launch Time":1427397549802,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3156,"Index":150,"Attempt":0,"Launch Time":1427397549802,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3147,"Index":141,"Attempt":0,"Launch Time":1427397549772,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549802,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":29,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":21818146,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3153,"Index":147,"Attempt":0,"Launch Time":1427397549794,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549802,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":360540,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3157,"Index":151,"Attempt":0,"Launch Time":1427397549803,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3154,"Index":148,"Attempt":0,"Launch Time":1427397549795,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549803,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":319638,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3158,"Index":152,"Attempt":0,"Launch Time":1427397549806,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3149,"Index":143,"Attempt":0,"Launch Time":1427397549774,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549806,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":31,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":401792,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3159,"Index":153,"Attempt":0,"Launch Time":1427397549810,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3156,"Index":150,"Attempt":0,"Launch Time":1427397549802,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549810,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":333382,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3160,"Index":154,"Attempt":0,"Launch Time":1427397549810,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3155,"Index":149,"Attempt":0,"Launch Time":1427397549802,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549810,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":311636,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3161,"Index":155,"Attempt":0,"Launch Time":1427397549810,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3157,"Index":151,"Attempt":0,"Launch Time":1427397549803,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549811,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":358117,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3162,"Index":156,"Attempt":0,"Launch Time":1427397549814,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3134,"Index":128,"Attempt":0,"Launch Time":1427397549738,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549814,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":52,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":29710089,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3163,"Index":157,"Attempt":0,"Launch Time":1427397549817,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3150,"Index":144,"Attempt":0,"Launch Time":1427397549778,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549818,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":340365,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3164,"Index":158,"Attempt":0,"Launch Time":1427397549819,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3160,"Index":154,"Attempt":0,"Launch Time":1427397549810,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549819,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":312659,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3165,"Index":159,"Attempt":0,"Launch Time":1427397549819,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3158,"Index":152,"Attempt":0,"Launch Time":1427397549806,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549819,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":368492,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3166,"Index":160,"Attempt":0,"Launch Time":1427397549820,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3161,"Index":155,"Attempt":0,"Launch Time":1427397549810,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549820,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":312596,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3167,"Index":161,"Attempt":0,"Launch Time":1427397549820,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3159,"Index":153,"Attempt":0,"Launch Time":1427397549810,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549820,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":326747,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3168,"Index":162,"Attempt":0,"Launch Time":1427397549824,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3162,"Index":156,"Attempt":0,"Launch Time":1427397549814,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549824,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":291682,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3169,"Index":163,"Attempt":0,"Launch Time":1427397549826,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3164,"Index":158,"Attempt":0,"Launch Time":1427397549819,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549826,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":305083,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3170,"Index":164,"Attempt":0,"Launch Time":1427397549826,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3167,"Index":161,"Attempt":0,"Launch Time":1427397549820,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549826,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":310247,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3171,"Index":165,"Attempt":0,"Launch Time":1427397549828,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3172,"Index":166,"Attempt":0,"Launch Time":1427397549828,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3165,"Index":159,"Attempt":0,"Launch Time":1427397549819,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549828,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":282415,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3166,"Index":160,"Attempt":0,"Launch Time":1427397549820,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549829,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1305606,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3173,"Index":167,"Attempt":0,"Launch Time":1427397549833,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3170,"Index":164,"Attempt":0,"Launch Time":1427397549826,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549833,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":328963,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3174,"Index":168,"Attempt":0,"Launch Time":1427397549834,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3169,"Index":163,"Attempt":0,"Launch Time":1427397549826,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549834,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":379741,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3175,"Index":169,"Attempt":0,"Launch Time":1427397549836,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3171,"Index":165,"Attempt":0,"Launch Time":1427397549828,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549836,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":314950,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3176,"Index":170,"Attempt":0,"Launch Time":1427397549837,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3172,"Index":166,"Attempt":0,"Launch Time":1427397549828,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549837,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":304597,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3177,"Index":171,"Attempt":0,"Launch Time":1427397549841,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3178,"Index":172,"Attempt":0,"Launch Time":1427397549841,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3173,"Index":167,"Attempt":0,"Launch Time":1427397549833,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549841,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":340883,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3174,"Index":168,"Attempt":0,"Launch Time":1427397549834,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549841,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":302741,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3179,"Index":173,"Attempt":0,"Launch Time":1427397549843,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3175,"Index":169,"Attempt":0,"Launch Time":1427397549836,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549843,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":362987,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3180,"Index":174,"Attempt":0,"Launch Time":1427397549844,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3168,"Index":162,"Attempt":0,"Launch Time":1427397549824,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549844,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":19,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1085204,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3181,"Index":175,"Attempt":0,"Launch Time":1427397549848,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3182,"Index":176,"Attempt":0,"Launch Time":1427397549848,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3178,"Index":172,"Attempt":0,"Launch Time":1427397549841,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549849,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":350605,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3177,"Index":171,"Attempt":0,"Launch Time":1427397549841,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549849,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":344053,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3183,"Index":177,"Attempt":0,"Launch Time":1427397549851,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3180,"Index":174,"Attempt":0,"Launch Time":1427397549844,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549851,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":307954,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3184,"Index":178,"Attempt":0,"Launch Time":1427397549851,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3179,"Index":173,"Attempt":0,"Launch Time":1427397549843,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549851,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":897796,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3185,"Index":179,"Attempt":0,"Launch Time":1427397549852,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3176,"Index":170,"Attempt":0,"Launch Time":1427397549837,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549852,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":358929,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3186,"Index":180,"Attempt":0,"Launch Time":1427397549852,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3163,"Index":157,"Attempt":0,"Launch Time":1427397549817,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549852,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":32,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":352920,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3187,"Index":181,"Attempt":0,"Launch Time":1427397549856,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3181,"Index":175,"Attempt":0,"Launch Time":1427397549848,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549856,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":330318,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3188,"Index":182,"Attempt":0,"Launch Time":1427397549856,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3182,"Index":176,"Attempt":0,"Launch Time":1427397549848,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549856,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":348315,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3189,"Index":183,"Attempt":0,"Launch Time":1427397549858,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3183,"Index":177,"Attempt":0,"Launch Time":1427397549851,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549858,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":311205,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3190,"Index":184,"Attempt":0,"Launch Time":1427397549858,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549866,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3185,"Index":179,"Attempt":0,"Launch Time":1427397549852,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549858,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":306204,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3191,"Index":185,"Attempt":0,"Launch Time":1427397549859,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3186,"Index":180,"Attempt":0,"Launch Time":1427397549852,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549859,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":320641,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3192,"Index":186,"Attempt":0,"Launch Time":1427397549859,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3124,"Index":118,"Attempt":0,"Launch Time":1427397549720,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549860,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":139,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":430162,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3193,"Index":187,"Attempt":0,"Launch Time":1427397549863,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3130,"Index":124,"Attempt":0,"Launch Time":1427397549732,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549863,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":68,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":315963,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3194,"Index":188,"Attempt":0,"Launch Time":1427397549865,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3190,"Index":184,"Attempt":0,"Launch Time":1427397549858,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549866,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":364252,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3195,"Index":189,"Attempt":0,"Launch Time":1427397549868,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3191,"Index":185,"Attempt":0,"Launch Time":1427397549859,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549869,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":407518,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3196,"Index":190,"Attempt":0,"Launch Time":1427397549872,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3192,"Index":186,"Attempt":0,"Launch Time":1427397549859,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549872,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":348024,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3197,"Index":191,"Attempt":0,"Launch Time":1427397549873,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3198,"Index":192,"Attempt":0,"Launch Time":1427397549873,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3193,"Index":187,"Attempt":0,"Launch Time":1427397549863,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549873,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":536036,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3194,"Index":188,"Attempt":0,"Launch Time":1427397549865,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549873,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":327410,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3199,"Index":193,"Attempt":0,"Launch Time":1427397549888,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3197,"Index":191,"Attempt":0,"Launch Time":1427397549873,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549888,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":315788,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3200,"Index":194,"Attempt":0,"Launch Time":1427397549889,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3195,"Index":189,"Attempt":0,"Launch Time":1427397549868,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549889,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":389696,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3201,"Index":195,"Attempt":0,"Launch Time":1427397549895,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3199,"Index":193,"Attempt":0,"Launch Time":1427397549888,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549896,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":330404,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3202,"Index":196,"Attempt":0,"Launch Time":1427397549896,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3200,"Index":194,"Attempt":0,"Launch Time":1427397549889,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549896,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":311690,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3203,"Index":197,"Attempt":0,"Launch Time":1427397549911,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3196,"Index":190,"Attempt":0,"Launch Time":1427397549872,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549911,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":38,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":342992,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3204,"Index":198,"Attempt":0,"Launch Time":1427397549913,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3198,"Index":192,"Attempt":0,"Launch Time":1427397549873,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549913,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":37,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":346766,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3205,"Index":199,"Attempt":0,"Launch Time":1427397549913,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3184,"Index":178,"Attempt":0,"Launch Time":1427397549851,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549913,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":56,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":350797,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3206,"Index":200,"Attempt":0,"Launch Time":1427397549914,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3207,"Index":201,"Attempt":0,"Launch Time":1427397549914,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3202,"Index":196,"Attempt":0,"Launch Time":1427397549896,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549914,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":277639,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3201,"Index":195,"Attempt":0,"Launch Time":1427397549895,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549914,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":331511,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3208,"Index":202,"Attempt":0,"Launch Time":1427397549923,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3204,"Index":198,"Attempt":0,"Launch Time":1427397549913,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549923,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":393547,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3209,"Index":203,"Attempt":0,"Launch Time":1427397549923,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3207,"Index":201,"Attempt":0,"Launch Time":1427397549914,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549924,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":322197,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3210,"Index":204,"Attempt":0,"Launch Time":1427397549924,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3203,"Index":197,"Attempt":0,"Launch Time":1427397549911,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549924,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":293232,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3211,"Index":205,"Attempt":0,"Launch Time":1427397549931,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3206,"Index":200,"Attempt":0,"Launch Time":1427397549914,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549931,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":327944,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3205,"Index":199,"Attempt":0,"Launch Time":1427397549913,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549931,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":335152,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3212,"Index":206,"Attempt":0,"Launch Time":1427397549931,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3213,"Index":207,"Attempt":0,"Launch Time":1427397549939,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3211,"Index":205,"Attempt":0,"Launch Time":1427397549931,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549939,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":475869,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3210,"Index":204,"Attempt":0,"Launch Time":1427397549924,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549940,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":319652,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3214,"Index":208,"Attempt":0,"Launch Time":1427397549940,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3215,"Index":209,"Attempt":0,"Launch Time":1427397549942,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3208,"Index":202,"Attempt":0,"Launch Time":1427397549923,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549942,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":435687,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3216,"Index":210,"Attempt":0,"Launch Time":1427397549943,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3209,"Index":203,"Attempt":0,"Launch Time":1427397549923,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549943,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":343322,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3217,"Index":211,"Attempt":0,"Launch Time":1427397549943,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3212,"Index":206,"Attempt":0,"Launch Time":1427397549931,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549943,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":351817,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3218,"Index":212,"Attempt":0,"Launch Time":1427397549946,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3213,"Index":207,"Attempt":0,"Launch Time":1427397549939,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549946,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":326374,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3219,"Index":213,"Attempt":0,"Launch Time":1427397549948,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3214,"Index":208,"Attempt":0,"Launch Time":1427397549940,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549948,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":355847,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3220,"Index":214,"Attempt":0,"Launch Time":1427397549949,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3215,"Index":209,"Attempt":0,"Launch Time":1427397549942,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549950,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":331841,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3221,"Index":215,"Attempt":0,"Launch Time":1427397549955,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3217,"Index":211,"Attempt":0,"Launch Time":1427397549943,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549955,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":417789,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3222,"Index":216,"Attempt":0,"Launch Time":1427397549956,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3218,"Index":212,"Attempt":0,"Launch Time":1427397549946,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549956,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":381636,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3223,"Index":217,"Attempt":0,"Launch Time":1427397549956,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3224,"Index":218,"Attempt":0,"Launch Time":1427397549956,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3225,"Index":219,"Attempt":0,"Launch Time":1427397549957,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3219,"Index":213,"Attempt":0,"Launch Time":1427397549948,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549957,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":354763,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3216,"Index":210,"Attempt":0,"Launch Time":1427397549943,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549957,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":411060,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3189,"Index":183,"Attempt":0,"Launch Time":1427397549858,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549957,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":40,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":414384,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3226,"Index":220,"Attempt":0,"Launch Time":1427397549957,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3220,"Index":214,"Attempt":0,"Launch Time":1427397549949,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549957,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":306548,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3227,"Index":221,"Attempt":0,"Launch Time":1427397549964,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3222,"Index":216,"Attempt":0,"Launch Time":1427397549956,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549964,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":741697,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3228,"Index":222,"Attempt":0,"Launch Time":1427397549964,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3224,"Index":218,"Attempt":0,"Launch Time":1427397549956,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549965,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":360073,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3229,"Index":223,"Attempt":0,"Launch Time":1427397549965,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3225,"Index":219,"Attempt":0,"Launch Time":1427397549957,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549966,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":742622,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3230,"Index":224,"Attempt":0,"Launch Time":1427397549967,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3223,"Index":217,"Attempt":0,"Launch Time":1427397549956,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549967,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1645111,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3231,"Index":225,"Attempt":0,"Launch Time":1427397549972,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3228,"Index":222,"Attempt":0,"Launch Time":1427397549964,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549972,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":330857,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3232,"Index":226,"Attempt":0,"Launch Time":1427397549972,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3229,"Index":223,"Attempt":0,"Launch Time":1427397549965,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549973,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":322680,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3233,"Index":227,"Attempt":0,"Launch Time":1427397549973,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3227,"Index":221,"Attempt":0,"Launch Time":1427397549964,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549973,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":587326,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3234,"Index":228,"Attempt":0,"Launch Time":1427397549974,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3226,"Index":220,"Attempt":0,"Launch Time":1427397549957,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549974,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":3312988,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3235,"Index":229,"Attempt":0,"Launch Time":1427397549975,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3230,"Index":224,"Attempt":0,"Launch Time":1427397549967,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549975,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":316513,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3236,"Index":230,"Attempt":0,"Launch Time":1427397549977,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3221,"Index":215,"Attempt":0,"Launch Time":1427397549955,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549978,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":22,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":726902,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3237,"Index":231,"Attempt":0,"Launch Time":1427397549979,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3231,"Index":225,"Attempt":0,"Launch Time":1427397549972,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549979,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":293069,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3238,"Index":232,"Attempt":0,"Launch Time":1427397549981,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3232,"Index":226,"Attempt":0,"Launch Time":1427397549972,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549981,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":466523,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3239,"Index":233,"Attempt":0,"Launch Time":1427397549983,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3234,"Index":228,"Attempt":0,"Launch Time":1427397549974,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549983,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":400249,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3240,"Index":234,"Attempt":0,"Launch Time":1427397549985,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3236,"Index":230,"Attempt":0,"Launch Time":1427397549977,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549985,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":338572,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3241,"Index":235,"Attempt":0,"Launch Time":1427397549985,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3188,"Index":182,"Attempt":0,"Launch Time":1427397549856,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549985,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":6,"Executor Run Time":96,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":344378,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3242,"Index":236,"Attempt":0,"Launch Time":1427397550004,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3187,"Index":181,"Attempt":0,"Launch Time":1427397549856,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550005,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":16,"Executor Run Time":88,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":314318,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3243,"Index":237,"Attempt":0,"Launch Time":1427397550015,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3240,"Index":234,"Attempt":0,"Launch Time":1427397549985,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550015,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":30,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":683417,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3244,"Index":238,"Attempt":0,"Launch Time":1427397550017,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3238,"Index":232,"Attempt":0,"Launch Time":1427397549981,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550017,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":8,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":427300,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3245,"Index":239,"Attempt":0,"Launch Time":1427397550017,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3246,"Index":240,"Attempt":0,"Launch Time":1427397550018,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3242,"Index":236,"Attempt":0,"Launch Time":1427397550004,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550018,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":6,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":474973,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3239,"Index":233,"Attempt":0,"Launch Time":1427397549983,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550018,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":8,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1436403,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3247,"Index":241,"Attempt":0,"Launch Time":1427397550021,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3243,"Index":237,"Attempt":0,"Launch Time":1427397550015,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550021,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":274435,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3248,"Index":242,"Attempt":0,"Launch Time":1427397550023,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3244,"Index":238,"Attempt":0,"Launch Time":1427397550017,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550023,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":280051,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3249,"Index":243,"Attempt":0,"Launch Time":1427397550025,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3245,"Index":239,"Attempt":0,"Launch Time":1427397550017,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550025,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":324481,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3250,"Index":244,"Attempt":0,"Launch Time":1427397550029,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3247,"Index":241,"Attempt":0,"Launch Time":1427397550021,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550029,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":319403,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3251,"Index":245,"Attempt":0,"Launch Time":1427397550029,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3246,"Index":240,"Attempt":0,"Launch Time":1427397550018,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550029,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":413743,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3252,"Index":246,"Attempt":0,"Launch Time":1427397550031,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3248,"Index":242,"Attempt":0,"Launch Time":1427397550023,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550031,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":316472,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3253,"Index":247,"Attempt":0,"Launch Time":1427397550031,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3249,"Index":243,"Attempt":0,"Launch Time":1427397550025,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550032,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":281688,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3254,"Index":248,"Attempt":0,"Launch Time":1427397550036,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3255,"Index":249,"Attempt":0,"Launch Time":1427397550036,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3250,"Index":244,"Attempt":0,"Launch Time":1427397550029,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550036,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":318347,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3251,"Index":245,"Attempt":0,"Launch Time":1427397550029,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550036,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":291954,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3256,"Index":250,"Attempt":0,"Launch Time":1427397550039,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3252,"Index":246,"Attempt":0,"Launch Time":1427397550031,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550040,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":404002,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3257,"Index":251,"Attempt":0,"Launch Time":1427397550044,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3258,"Index":252,"Attempt":0,"Launch Time":1427397550044,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3254,"Index":248,"Attempt":0,"Launch Time":1427397550036,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550044,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":347800,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3253,"Index":247,"Attempt":0,"Launch Time":1427397550031,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550044,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1004819,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3259,"Index":253,"Attempt":0,"Launch Time":1427397550045,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3255,"Index":249,"Attempt":0,"Launch Time":1427397550036,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550045,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9613,"Shuffle Write Time":362739,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3260,"Index":254,"Attempt":0,"Launch Time":1427397550048,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3241,"Index":235,"Attempt":0,"Launch Time":1427397549985,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550048,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":39,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":3548090,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3261,"Index":255,"Attempt":0,"Launch Time":1427397550053,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3258,"Index":252,"Attempt":0,"Launch Time":1427397550044,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550053,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":409182,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3262,"Index":256,"Attempt":0,"Launch Time":1427397550053,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3259,"Index":253,"Attempt":0,"Launch Time":1427397550045,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550053,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":361049,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3263,"Index":257,"Attempt":0,"Launch Time":1427397550055,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3256,"Index":250,"Attempt":0,"Launch Time":1427397550039,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550055,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":497293,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3264,"Index":258,"Attempt":0,"Launch Time":1427397550056,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3257,"Index":251,"Attempt":0,"Launch Time":1427397550044,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550056,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1120524,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3265,"Index":259,"Attempt":0,"Launch Time":1427397550061,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3262,"Index":256,"Attempt":0,"Launch Time":1427397550053,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550061,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":321400,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3266,"Index":260,"Attempt":0,"Launch Time":1427397550061,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3261,"Index":255,"Attempt":0,"Launch Time":1427397550053,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550061,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":315574,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3267,"Index":261,"Attempt":0,"Launch Time":1427397550062,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3263,"Index":257,"Attempt":0,"Launch Time":1427397550055,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550062,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9613,"Shuffle Write Time":322093,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3268,"Index":262,"Attempt":0,"Launch Time":1427397550066,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3264,"Index":258,"Attempt":0,"Launch Time":1427397550056,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550066,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":317587,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3269,"Index":263,"Attempt":0,"Launch Time":1427397550068,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3265,"Index":259,"Attempt":0,"Launch Time":1427397550061,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550068,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":333946,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3270,"Index":264,"Attempt":0,"Launch Time":1427397550068,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3266,"Index":260,"Attempt":0,"Launch Time":1427397550061,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550068,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":317089,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3271,"Index":265,"Attempt":0,"Launch Time":1427397550069,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3235,"Index":229,"Attempt":0,"Launch Time":1427397549975,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550069,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":94,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":365642,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3272,"Index":266,"Attempt":0,"Launch Time":1427397550070,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3267,"Index":261,"Attempt":0,"Launch Time":1427397550062,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550070,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":336375,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3273,"Index":267,"Attempt":0,"Launch Time":1427397550073,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3260,"Index":254,"Attempt":0,"Launch Time":1427397550048,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550073,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":23,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":350737,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3274,"Index":268,"Attempt":0,"Launch Time":1427397550075,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3269,"Index":263,"Attempt":0,"Launch Time":1427397550068,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550075,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":336010,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3275,"Index":269,"Attempt":0,"Launch Time":1427397550075,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3268,"Index":262,"Attempt":0,"Launch Time":1427397550066,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550076,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":350233,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3276,"Index":270,"Attempt":0,"Launch Time":1427397550076,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3237,"Index":231,"Attempt":0,"Launch Time":1427397549979,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550076,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":97,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":574497,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3277,"Index":271,"Attempt":0,"Launch Time":1427397550077,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3272,"Index":266,"Attempt":0,"Launch Time":1427397550070,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550077,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":303801,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3278,"Index":272,"Attempt":0,"Launch Time":1427397550081,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3273,"Index":267,"Attempt":0,"Launch Time":1427397550073,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550081,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":384061,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3279,"Index":273,"Attempt":0,"Launch Time":1427397550083,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3274,"Index":268,"Attempt":0,"Launch Time":1427397550075,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550083,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":396011,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3280,"Index":274,"Attempt":0,"Launch Time":1427397550083,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3270,"Index":264,"Attempt":0,"Launch Time":1427397550068,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550083,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":5326852,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3281,"Index":275,"Attempt":0,"Launch Time":1427397550084,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3275,"Index":269,"Attempt":0,"Launch Time":1427397550075,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550084,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":388499,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3282,"Index":276,"Attempt":0,"Launch Time":1427397550086,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3277,"Index":271,"Attempt":0,"Launch Time":1427397550077,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550086,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":352432,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3283,"Index":277,"Attempt":0,"Launch Time":1427397550088,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3271,"Index":265,"Attempt":0,"Launch Time":1427397550069,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550088,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9613,"Shuffle Write Time":333904,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3284,"Index":278,"Attempt":0,"Launch Time":1427397550089,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3278,"Index":272,"Attempt":0,"Launch Time":1427397550081,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550089,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":335534,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3285,"Index":279,"Attempt":0,"Launch Time":1427397550090,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3279,"Index":273,"Attempt":0,"Launch Time":1427397550083,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550090,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":329420,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3286,"Index":280,"Attempt":0,"Launch Time":1427397550091,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3280,"Index":274,"Attempt":0,"Launch Time":1427397550083,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550091,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":335357,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3287,"Index":281,"Attempt":0,"Launch Time":1427397550105,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3281,"Index":275,"Attempt":0,"Launch Time":1427397550084,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550105,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":20,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":326339,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3288,"Index":282,"Attempt":0,"Launch Time":1427397550106,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3233,"Index":227,"Attempt":0,"Launch Time":1427397549973,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550106,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":131,"Result Size":930,"JVM GC Time":16,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":101411330,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3289,"Index":283,"Attempt":0,"Launch Time":1427397550107,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3276,"Index":270,"Attempt":0,"Launch Time":1427397550076,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550107,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":31,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":633627,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3290,"Index":284,"Attempt":0,"Launch Time":1427397550109,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3286,"Index":280,"Attempt":0,"Launch Time":1427397550091,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550109,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":266678,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3291,"Index":285,"Attempt":0,"Launch Time":1427397550112,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3283,"Index":277,"Attempt":0,"Launch Time":1427397550088,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550112,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":14,"Executor Run Time":7,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":316164,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3292,"Index":286,"Attempt":0,"Launch Time":1427397550113,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3288,"Index":282,"Attempt":0,"Launch Time":1427397550106,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550113,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":309869,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3293,"Index":287,"Attempt":0,"Launch Time":1427397550116,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3289,"Index":283,"Attempt":0,"Launch Time":1427397550107,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550116,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":359646,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3294,"Index":288,"Attempt":0,"Launch Time":1427397550121,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3292,"Index":286,"Attempt":0,"Launch Time":1427397550113,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550121,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":316105,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3295,"Index":289,"Attempt":0,"Launch Time":1427397550122,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3291,"Index":285,"Attempt":0,"Launch Time":1427397550112,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550122,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":367383,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3296,"Index":290,"Attempt":0,"Launch Time":1427397550124,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3285,"Index":279,"Attempt":0,"Launch Time":1427397550090,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550124,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":33,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":392047,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3297,"Index":291,"Attempt":0,"Launch Time":1427397550127,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3293,"Index":287,"Attempt":0,"Launch Time":1427397550116,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550127,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":348634,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3298,"Index":292,"Attempt":0,"Launch Time":1427397550130,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3294,"Index":288,"Attempt":0,"Launch Time":1427397550121,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550131,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1258984,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3299,"Index":293,"Attempt":0,"Launch Time":1427397550132,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3295,"Index":289,"Attempt":0,"Launch Time":1427397550122,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550132,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":2267281,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3300,"Index":294,"Attempt":0,"Launch Time":1427397550134,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3297,"Index":291,"Attempt":0,"Launch Time":1427397550127,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550134,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":350417,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3301,"Index":295,"Attempt":0,"Launch Time":1427397550136,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3296,"Index":290,"Attempt":0,"Launch Time":1427397550124,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550136,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":294689,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3302,"Index":296,"Attempt":0,"Launch Time":1427397550137,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3298,"Index":292,"Attempt":0,"Launch Time":1427397550130,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550137,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":328747,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3303,"Index":297,"Attempt":0,"Launch Time":1427397550138,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3284,"Index":278,"Attempt":0,"Launch Time":1427397550089,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550138,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":48,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":317502,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3304,"Index":298,"Attempt":0,"Launch Time":1427397550139,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3299,"Index":293,"Attempt":0,"Launch Time":1427397550132,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550139,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":321300,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3305,"Index":299,"Attempt":0,"Launch Time":1427397550141,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3300,"Index":294,"Attempt":0,"Launch Time":1427397550134,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550141,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":289784,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3306,"Index":300,"Attempt":0,"Launch Time":1427397550144,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3302,"Index":296,"Attempt":0,"Launch Time":1427397550137,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550144,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":318419,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3307,"Index":301,"Attempt":0,"Launch Time":1427397550145,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3304,"Index":298,"Attempt":0,"Launch Time":1427397550139,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550146,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":316504,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3308,"Index":302,"Attempt":0,"Launch Time":1427397550147,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3301,"Index":295,"Attempt":0,"Launch Time":1427397550136,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550147,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":325189,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3309,"Index":303,"Attempt":0,"Launch Time":1427397550149,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3305,"Index":299,"Attempt":0,"Launch Time":1427397550141,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550150,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":419631,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3310,"Index":304,"Attempt":0,"Launch Time":1427397550151,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3306,"Index":300,"Attempt":0,"Launch Time":1427397550144,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550151,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":274335,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3311,"Index":305,"Attempt":0,"Launch Time":1427397550153,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3307,"Index":301,"Attempt":0,"Launch Time":1427397550145,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550153,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":361918,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3312,"Index":306,"Attempt":0,"Launch Time":1427397550154,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3308,"Index":302,"Attempt":0,"Launch Time":1427397550147,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550154,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":269776,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3313,"Index":307,"Attempt":0,"Launch Time":1427397550156,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3309,"Index":303,"Attempt":0,"Launch Time":1427397550149,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550156,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":274199,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3314,"Index":308,"Attempt":0,"Launch Time":1427397550158,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3303,"Index":297,"Attempt":0,"Launch Time":1427397550138,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550158,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":716078,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3315,"Index":309,"Attempt":0,"Launch Time":1427397550158,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3310,"Index":304,"Attempt":0,"Launch Time":1427397550151,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550159,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":293157,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3316,"Index":310,"Attempt":0,"Launch Time":1427397550160,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3311,"Index":305,"Attempt":0,"Launch Time":1427397550153,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550160,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":308353,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3317,"Index":311,"Attempt":0,"Launch Time":1427397550162,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3312,"Index":306,"Attempt":0,"Launch Time":1427397550154,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550162,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":318115,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3318,"Index":312,"Attempt":0,"Launch Time":1427397550167,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3315,"Index":309,"Attempt":0,"Launch Time":1427397550158,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550167,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":1270519,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3319,"Index":313,"Attempt":0,"Launch Time":1427397550171,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3313,"Index":307,"Attempt":0,"Launch Time":1427397550156,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550171,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":402941,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3320,"Index":314,"Attempt":0,"Launch Time":1427397550173,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3317,"Index":311,"Attempt":0,"Launch Time":1427397550162,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550173,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":354674,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3321,"Index":315,"Attempt":0,"Launch Time":1427397550173,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3316,"Index":310,"Attempt":0,"Launch Time":1427397550160,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550174,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":354958,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3322,"Index":316,"Attempt":0,"Launch Time":1427397550174,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3318,"Index":312,"Attempt":0,"Launch Time":1427397550167,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550174,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":329908,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3323,"Index":317,"Attempt":0,"Launch Time":1427397550178,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3319,"Index":313,"Attempt":0,"Launch Time":1427397550171,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550178,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":344289,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3324,"Index":318,"Attempt":0,"Launch Time":1427397550181,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3322,"Index":316,"Attempt":0,"Launch Time":1427397550174,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550182,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":350866,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3325,"Index":319,"Attempt":0,"Launch Time":1427397550182,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3320,"Index":314,"Attempt":0,"Launch Time":1427397550173,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550182,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":363824,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3326,"Index":320,"Attempt":0,"Launch Time":1427397550182,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3321,"Index":315,"Attempt":0,"Launch Time":1427397550173,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550182,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":428159,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3327,"Index":321,"Attempt":0,"Launch Time":1427397550185,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3314,"Index":308,"Attempt":0,"Launch Time":1427397550158,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550185,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":337624,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3328,"Index":322,"Attempt":0,"Launch Time":1427397550185,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3323,"Index":317,"Attempt":0,"Launch Time":1427397550178,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550185,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":325426,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3329,"Index":323,"Attempt":0,"Launch Time":1427397550191,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3324,"Index":318,"Attempt":0,"Launch Time":1427397550181,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550191,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":402708,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3330,"Index":324,"Attempt":0,"Launch Time":1427397550192,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3326,"Index":320,"Attempt":0,"Launch Time":1427397550182,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550193,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":493205,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3331,"Index":325,"Attempt":0,"Launch Time":1427397550201,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3325,"Index":319,"Attempt":0,"Launch Time":1427397550182,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550201,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":471053,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3332,"Index":326,"Attempt":0,"Launch Time":1427397550208,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3331,"Index":325,"Attempt":0,"Launch Time":1427397550201,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550209,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":326431,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3333,"Index":327,"Attempt":0,"Launch Time":1427397550216,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3332,"Index":326,"Attempt":0,"Launch Time":1427397550208,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550216,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":349302,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3334,"Index":328,"Attempt":0,"Launch Time":1427397550220,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3328,"Index":322,"Attempt":0,"Launch Time":1427397550185,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550220,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":33,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":326474,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3335,"Index":329,"Attempt":0,"Launch Time":1427397550220,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3327,"Index":321,"Attempt":0,"Launch Time":1427397550185,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550220,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":35,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":333826,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3336,"Index":330,"Attempt":0,"Launch Time":1427397550224,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3337,"Index":331,"Attempt":0,"Launch Time":1427397550225,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3338,"Index":332,"Attempt":0,"Launch Time":1427397550225,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3329,"Index":323,"Attempt":0,"Launch Time":1427397550191,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550225,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":31,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":351852,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3333,"Index":327,"Attempt":0,"Launch Time":1427397550216,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550226,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":374930,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3330,"Index":324,"Attempt":0,"Launch Time":1427397550192,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550226,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":30,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":312912,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3339,"Index":333,"Attempt":0,"Launch Time":1427397550228,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3334,"Index":328,"Attempt":0,"Launch Time":1427397550220,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550228,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":341295,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3335,"Index":329,"Attempt":0,"Launch Time":1427397550220,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550229,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":348756,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3340,"Index":334,"Attempt":0,"Launch Time":1427397550229,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3341,"Index":335,"Attempt":0,"Launch Time":1427397550232,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3337,"Index":331,"Attempt":0,"Launch Time":1427397550225,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550233,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":454506,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3342,"Index":336,"Attempt":0,"Launch Time":1427397550233,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3336,"Index":330,"Attempt":0,"Launch Time":1427397550224,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550233,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":385922,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3343,"Index":337,"Attempt":0,"Launch Time":1427397550244,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3344,"Index":338,"Attempt":0,"Launch Time":1427397550244,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3287,"Index":281,"Attempt":0,"Launch Time":1427397550105,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550244,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":17,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":324865,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3282,"Index":276,"Attempt":0,"Launch Time":1427397550086,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550244,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":148,"Result Size":930,"JVM GC Time":15,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":976102,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3345,"Index":339,"Attempt":0,"Launch Time":1427397550245,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3290,"Index":284,"Attempt":0,"Launch Time":1427397550109,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550245,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":21,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":298273,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3346,"Index":340,"Attempt":0,"Launch Time":1427397550251,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3344,"Index":338,"Attempt":0,"Launch Time":1427397550244,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550251,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9651,"Shuffle Write Time":320858,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3347,"Index":341,"Attempt":0,"Launch Time":1427397550251,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3343,"Index":337,"Attempt":0,"Launch Time":1427397550244,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550251,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9651,"Shuffle Write Time":333484,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3348,"Index":342,"Attempt":0,"Launch Time":1427397550258,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3346,"Index":340,"Attempt":0,"Launch Time":1427397550251,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550258,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9649,"Shuffle Write Time":331943,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3349,"Index":343,"Attempt":0,"Launch Time":1427397550258,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3347,"Index":341,"Attempt":0,"Launch Time":1427397550251,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550259,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":356093,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3350,"Index":344,"Attempt":0,"Launch Time":1427397550259,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3345,"Index":339,"Attempt":0,"Launch Time":1427397550245,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550259,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9647,"Shuffle Write Time":573406,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3351,"Index":345,"Attempt":0,"Launch Time":1427397550265,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3349,"Index":343,"Attempt":0,"Launch Time":1427397550258,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550265,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":331414,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3352,"Index":346,"Attempt":0,"Launch Time":1427397550266,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3348,"Index":342,"Attempt":0,"Launch Time":1427397550258,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550266,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":327827,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3353,"Index":347,"Attempt":0,"Launch Time":1427397550267,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3350,"Index":344,"Attempt":0,"Launch Time":1427397550259,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550267,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":336828,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3354,"Index":348,"Attempt":0,"Launch Time":1427397550273,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3340,"Index":334,"Attempt":0,"Launch Time":1427397550229,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550273,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":16,"Executor Run Time":27,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9666,"Shuffle Write Time":346703,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3355,"Index":349,"Attempt":0,"Launch Time":1427397550273,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3351,"Index":345,"Attempt":0,"Launch Time":1427397550265,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550273,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":377902,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3356,"Index":350,"Attempt":0,"Launch Time":1427397550273,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3352,"Index":346,"Attempt":0,"Launch Time":1427397550266,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550274,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":332368,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3357,"Index":351,"Attempt":0,"Launch Time":1427397550278,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3353,"Index":347,"Attempt":0,"Launch Time":1427397550267,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550278,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":591338,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3358,"Index":352,"Attempt":0,"Launch Time":1427397550282,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3355,"Index":349,"Attempt":0,"Launch Time":1427397550273,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550282,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":468108,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3359,"Index":353,"Attempt":0,"Launch Time":1427397550283,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3360,"Index":354,"Attempt":0,"Launch Time":1427397550283,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3356,"Index":350,"Attempt":0,"Launch Time":1427397550273,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550283,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":471297,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3354,"Index":348,"Attempt":0,"Launch Time":1427397550273,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550283,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1061149,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3361,"Index":355,"Attempt":0,"Launch Time":1427397550284,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3342,"Index":336,"Attempt":0,"Launch Time":1427397550233,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550284,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9648,"Shuffle Write Time":422100,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3362,"Index":356,"Attempt":0,"Launch Time":1427397550286,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3357,"Index":351,"Attempt":0,"Launch Time":1427397550278,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550286,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":298004,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3363,"Index":357,"Attempt":0,"Launch Time":1427397550290,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3360,"Index":354,"Attempt":0,"Launch Time":1427397550283,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550290,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":321176,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3364,"Index":358,"Attempt":0,"Launch Time":1427397550291,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3358,"Index":352,"Attempt":0,"Launch Time":1427397550282,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550291,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":336461,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3365,"Index":359,"Attempt":0,"Launch Time":1427397550293,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3361,"Index":355,"Attempt":0,"Launch Time":1427397550284,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550294,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":333781,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3366,"Index":360,"Attempt":0,"Launch Time":1427397550295,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3362,"Index":356,"Attempt":0,"Launch Time":1427397550286,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550295,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":467129,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3367,"Index":361,"Attempt":0,"Launch Time":1427397550298,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3359,"Index":353,"Attempt":0,"Launch Time":1427397550283,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550298,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":394522,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3368,"Index":362,"Attempt":0,"Launch Time":1427397550300,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3364,"Index":358,"Attempt":0,"Launch Time":1427397550291,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550300,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":787625,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3369,"Index":363,"Attempt":0,"Launch Time":1427397550301,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3365,"Index":359,"Attempt":0,"Launch Time":1427397550293,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550301,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":344976,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3370,"Index":364,"Attempt":0,"Launch Time":1427397550301,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3363,"Index":357,"Attempt":0,"Launch Time":1427397550290,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550301,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":386484,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3371,"Index":365,"Attempt":0,"Launch Time":1427397550307,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3366,"Index":360,"Attempt":0,"Launch Time":1427397550295,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550308,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":370597,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3372,"Index":366,"Attempt":0,"Launch Time":1427397550308,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3368,"Index":362,"Attempt":0,"Launch Time":1427397550300,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550308,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":339205,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3373,"Index":367,"Attempt":0,"Launch Time":1427397550309,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3370,"Index":364,"Attempt":0,"Launch Time":1427397550301,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550309,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":366876,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3374,"Index":368,"Attempt":0,"Launch Time":1427397550310,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3369,"Index":363,"Attempt":0,"Launch Time":1427397550301,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550310,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":373115,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3375,"Index":369,"Attempt":0,"Launch Time":1427397550312,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3367,"Index":361,"Attempt":0,"Launch Time":1427397550298,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550318,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":400941,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3376,"Index":370,"Attempt":0,"Launch Time":1427397550320,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3338,"Index":332,"Attempt":0,"Launch Time":1427397550225,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550320,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":91,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":397788,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3377,"Index":371,"Attempt":0,"Launch Time":1427397550323,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3372,"Index":366,"Attempt":0,"Launch Time":1427397550308,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550324,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":342054,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3378,"Index":372,"Attempt":0,"Launch Time":1427397550325,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3375,"Index":369,"Attempt":0,"Launch Time":1427397550312,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550325,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":298023,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3379,"Index":373,"Attempt":0,"Launch Time":1427397550326,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3373,"Index":367,"Attempt":0,"Launch Time":1427397550309,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550326,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":16,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":287637,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3380,"Index":374,"Attempt":0,"Launch Time":1427397550326,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3374,"Index":368,"Attempt":0,"Launch Time":1427397550310,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550326,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":315529,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3381,"Index":375,"Attempt":0,"Launch Time":1427397550330,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3371,"Index":365,"Attempt":0,"Launch Time":1427397550307,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550331,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":22,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":8857312,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3382,"Index":376,"Attempt":0,"Launch Time":1427397550333,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3377,"Index":371,"Attempt":0,"Launch Time":1427397550323,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550333,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":355138,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3383,"Index":377,"Attempt":0,"Launch Time":1427397550333,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3379,"Index":373,"Attempt":0,"Launch Time":1427397550326,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550333,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":337586,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3384,"Index":378,"Attempt":0,"Launch Time":1427397550334,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3380,"Index":374,"Attempt":0,"Launch Time":1427397550326,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550334,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9612,"Shuffle Write Time":385951,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3385,"Index":379,"Attempt":0,"Launch Time":1427397550336,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3378,"Index":372,"Attempt":0,"Launch Time":1427397550325,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550336,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":321656,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3386,"Index":380,"Attempt":0,"Launch Time":1427397550338,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3339,"Index":333,"Attempt":0,"Launch Time":1427397550228,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550338,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":109,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":333469,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3387,"Index":381,"Attempt":0,"Launch Time":1427397550338,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3381,"Index":375,"Attempt":0,"Launch Time":1427397550330,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550339,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":316651,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3388,"Index":382,"Attempt":0,"Launch Time":1427397550341,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3389,"Index":383,"Attempt":0,"Launch Time":1427397550341,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3390,"Index":384,"Attempt":0,"Launch Time":1427397550341,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3382,"Index":376,"Attempt":0,"Launch Time":1427397550333,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550341,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":325651,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3383,"Index":377,"Attempt":0,"Launch Time":1427397550333,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550341,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":322022,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3384,"Index":378,"Attempt":0,"Launch Time":1427397550334,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550342,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":340512,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3391,"Index":385,"Attempt":0,"Launch Time":1427397550346,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3386,"Index":380,"Attempt":0,"Launch Time":1427397550338,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550346,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":329511,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3392,"Index":386,"Attempt":0,"Launch Time":1427397550346,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3385,"Index":379,"Attempt":0,"Launch Time":1427397550336,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550347,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":328344,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3393,"Index":387,"Attempt":0,"Launch Time":1427397550349,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3388,"Index":382,"Attempt":0,"Launch Time":1427397550341,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550349,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9614,"Shuffle Write Time":392539,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3394,"Index":388,"Attempt":0,"Launch Time":1427397550350,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3390,"Index":384,"Attempt":0,"Launch Time":1427397550341,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550350,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":389745,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3395,"Index":389,"Attempt":0,"Launch Time":1427397550351,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3389,"Index":383,"Attempt":0,"Launch Time":1427397550341,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550351,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":409651,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3396,"Index":390,"Attempt":0,"Launch Time":1427397550353,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3391,"Index":385,"Attempt":0,"Launch Time":1427397550346,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550354,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":291786,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3397,"Index":391,"Attempt":0,"Launch Time":1427397550356,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3393,"Index":387,"Attempt":0,"Launch Time":1427397550349,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550356,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":281184,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3398,"Index":392,"Attempt":0,"Launch Time":1427397550358,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3395,"Index":389,"Attempt":0,"Launch Time":1427397550351,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550358,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":292691,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3399,"Index":393,"Attempt":0,"Launch Time":1427397550361,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3387,"Index":381,"Attempt":0,"Launch Time":1427397550338,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550361,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":275604,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3400,"Index":394,"Attempt":0,"Launch Time":1427397550368,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3376,"Index":370,"Attempt":0,"Launch Time":1427397550320,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550368,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":941138,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3401,"Index":395,"Attempt":0,"Launch Time":1427397550369,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3399,"Index":393,"Attempt":0,"Launch Time":1427397550361,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550369,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":304358,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3402,"Index":396,"Attempt":0,"Launch Time":1427397550369,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3341,"Index":335,"Attempt":0,"Launch Time":1427397550232,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550369,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":50,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9650,"Shuffle Write Time":332089,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3403,"Index":397,"Attempt":0,"Launch Time":1427397550376,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3404,"Index":398,"Attempt":0,"Launch Time":1427397550376,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3401,"Index":395,"Attempt":0,"Launch Time":1427397550369,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550376,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":334416,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3400,"Index":394,"Attempt":0,"Launch Time":1427397550368,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550376,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":334696,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3405,"Index":399,"Attempt":0,"Launch Time":1427397550379,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3402,"Index":396,"Attempt":0,"Launch Time":1427397550369,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550379,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":361871,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3406,"Index":400,"Attempt":0,"Launch Time":1427397550383,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3403,"Index":397,"Attempt":0,"Launch Time":1427397550376,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550383,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":365980,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3407,"Index":401,"Attempt":0,"Launch Time":1427397550384,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3404,"Index":398,"Attempt":0,"Launch Time":1427397550376,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550384,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":364681,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3408,"Index":402,"Attempt":0,"Launch Time":1427397550388,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3405,"Index":399,"Attempt":0,"Launch Time":1427397550379,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550388,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":306424,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3409,"Index":403,"Attempt":0,"Launch Time":1427397550390,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3406,"Index":400,"Attempt":0,"Launch Time":1427397550383,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550390,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":296309,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3410,"Index":404,"Attempt":0,"Launch Time":1427397550392,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3407,"Index":401,"Attempt":0,"Launch Time":1427397550384,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550392,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":308211,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3411,"Index":405,"Attempt":0,"Launch Time":1427397550395,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3408,"Index":402,"Attempt":0,"Launch Time":1427397550388,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550395,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":349027,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3412,"Index":406,"Attempt":0,"Launch Time":1427397550397,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3409,"Index":403,"Attempt":0,"Launch Time":1427397550390,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550397,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":356144,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3413,"Index":407,"Attempt":0,"Launch Time":1427397550399,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3410,"Index":404,"Attempt":0,"Launch Time":1427397550392,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550399,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":327321,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3414,"Index":408,"Attempt":0,"Launch Time":1427397550404,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3412,"Index":406,"Attempt":0,"Launch Time":1427397550397,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550404,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":328929,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3415,"Index":409,"Attempt":0,"Launch Time":1427397550405,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3411,"Index":405,"Attempt":0,"Launch Time":1427397550395,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550406,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":360399,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3416,"Index":410,"Attempt":0,"Launch Time":1427397550406,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3413,"Index":407,"Attempt":0,"Launch Time":1427397550399,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550406,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":324615,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3417,"Index":411,"Attempt":0,"Launch Time":1427397550411,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3414,"Index":408,"Attempt":0,"Launch Time":1427397550404,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550411,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":337274,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3418,"Index":412,"Attempt":0,"Launch Time":1427397550413,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3415,"Index":409,"Attempt":0,"Launch Time":1427397550405,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550413,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":353917,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3419,"Index":413,"Attempt":0,"Launch Time":1427397550413,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3416,"Index":410,"Attempt":0,"Launch Time":1427397550406,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550413,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":338657,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3420,"Index":414,"Attempt":0,"Launch Time":1427397550425,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3421,"Index":415,"Attempt":0,"Launch Time":1427397550426,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3418,"Index":412,"Attempt":0,"Launch Time":1427397550413,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550426,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":367342,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3417,"Index":411,"Attempt":0,"Launch Time":1427397550411,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550426,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":323126,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3422,"Index":416,"Attempt":0,"Launch Time":1427397550428,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3419,"Index":413,"Attempt":0,"Launch Time":1427397550413,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550428,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":358805,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3423,"Index":417,"Attempt":0,"Launch Time":1427397550430,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3392,"Index":386,"Attempt":0,"Launch Time":1427397550346,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550431,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":80,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":781620,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3424,"Index":418,"Attempt":0,"Launch Time":1427397550434,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3420,"Index":414,"Attempt":0,"Launch Time":1427397550425,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550435,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":420273,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3425,"Index":419,"Attempt":0,"Launch Time":1427397550435,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3421,"Index":415,"Attempt":0,"Launch Time":1427397550426,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550435,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":421478,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3426,"Index":420,"Attempt":0,"Launch Time":1427397550436,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3422,"Index":416,"Attempt":0,"Launch Time":1427397550428,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550436,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":369531,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3427,"Index":421,"Attempt":0,"Launch Time":1427397550440,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3423,"Index":417,"Attempt":0,"Launch Time":1427397550430,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550440,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":403698,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3428,"Index":422,"Attempt":0,"Launch Time":1427397550442,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3425,"Index":419,"Attempt":0,"Launch Time":1427397550435,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550443,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":339985,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3424,"Index":418,"Attempt":0,"Launch Time":1427397550434,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550443,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":313304,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3429,"Index":423,"Attempt":0,"Launch Time":1427397550443,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3430,"Index":424,"Attempt":0,"Launch Time":1427397550445,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3426,"Index":420,"Attempt":0,"Launch Time":1427397550436,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550445,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":506662,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3431,"Index":425,"Attempt":0,"Launch Time":1427397550446,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3427,"Index":421,"Attempt":0,"Launch Time":1427397550440,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550447,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":334616,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3432,"Index":426,"Attempt":0,"Launch Time":1427397550449,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3428,"Index":422,"Attempt":0,"Launch Time":1427397550442,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550450,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":311083,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3433,"Index":427,"Attempt":0,"Launch Time":1427397550450,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3429,"Index":423,"Attempt":0,"Launch Time":1427397550443,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550451,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":352772,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3434,"Index":428,"Attempt":0,"Launch Time":1427397550452,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3430,"Index":424,"Attempt":0,"Launch Time":1427397550445,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550452,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":303301,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3435,"Index":429,"Attempt":0,"Launch Time":1427397550452,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3431,"Index":425,"Attempt":0,"Launch Time":1427397550446,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550453,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":259006,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3436,"Index":430,"Attempt":0,"Launch Time":1427397550457,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3433,"Index":427,"Attempt":0,"Launch Time":1427397550450,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550457,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":314581,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3437,"Index":431,"Attempt":0,"Launch Time":1427397550458,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3434,"Index":428,"Attempt":0,"Launch Time":1427397550452,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550458,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":296967,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3438,"Index":432,"Attempt":0,"Launch Time":1427397550458,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3432,"Index":426,"Attempt":0,"Launch Time":1427397550449,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550459,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":276594,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3439,"Index":433,"Attempt":0,"Launch Time":1427397550459,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3435,"Index":429,"Attempt":0,"Launch Time":1427397550452,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550459,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":319441,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3440,"Index":434,"Attempt":0,"Launch Time":1427397550463,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3436,"Index":430,"Attempt":0,"Launch Time":1427397550457,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550463,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":313742,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3441,"Index":435,"Attempt":0,"Launch Time":1427397550466,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3438,"Index":432,"Attempt":0,"Launch Time":1427397550458,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550466,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":328956,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3442,"Index":436,"Attempt":0,"Launch Time":1427397550466,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3437,"Index":431,"Attempt":0,"Launch Time":1427397550458,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550466,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":300089,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3443,"Index":437,"Attempt":0,"Launch Time":1427397550470,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3439,"Index":433,"Attempt":0,"Launch Time":1427397550459,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550471,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":339252,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3444,"Index":438,"Attempt":0,"Launch Time":1427397550471,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3440,"Index":434,"Attempt":0,"Launch Time":1427397550463,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550471,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":329795,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3445,"Index":439,"Attempt":0,"Launch Time":1427397550472,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3398,"Index":392,"Attempt":0,"Launch Time":1427397550358,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550473,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":45,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1675806,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3446,"Index":440,"Attempt":0,"Launch Time":1427397550473,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3441,"Index":435,"Attempt":0,"Launch Time":1427397550466,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550473,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":327600,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3447,"Index":441,"Attempt":0,"Launch Time":1427397550474,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3442,"Index":436,"Attempt":0,"Launch Time":1427397550466,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550474,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":430470,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3448,"Index":442,"Attempt":0,"Launch Time":1427397550478,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3443,"Index":437,"Attempt":0,"Launch Time":1427397550470,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550478,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":406632,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3449,"Index":443,"Attempt":0,"Launch Time":1427397550479,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3444,"Index":438,"Attempt":0,"Launch Time":1427397550471,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550480,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":407969,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3450,"Index":444,"Attempt":0,"Launch Time":1427397550481,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3451,"Index":445,"Attempt":0,"Launch Time":1427397550481,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3396,"Index":390,"Attempt":0,"Launch Time":1427397550353,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550481,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":84,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9615,"Shuffle Write Time":443340,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3446,"Index":440,"Attempt":0,"Launch Time":1427397550473,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550481,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":455765,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3452,"Index":446,"Attempt":0,"Launch Time":1427397550483,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3447,"Index":441,"Attempt":0,"Launch Time":1427397550474,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550483,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":418697,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3453,"Index":447,"Attempt":0,"Launch Time":1427397550488,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3454,"Index":448,"Attempt":0,"Launch Time":1427397550489,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3394,"Index":388,"Attempt":0,"Launch Time":1427397550350,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550489,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":138,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":60231827,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3445,"Index":439,"Attempt":0,"Launch Time":1427397550472,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550489,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":390154,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3455,"Index":449,"Attempt":0,"Launch Time":1427397550490,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3450,"Index":444,"Attempt":0,"Launch Time":1427397550481,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550490,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":302421,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3456,"Index":450,"Attempt":0,"Launch Time":1427397550495,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3453,"Index":447,"Attempt":0,"Launch Time":1427397550488,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550495,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":307994,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3457,"Index":451,"Attempt":0,"Launch Time":1427397550496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3454,"Index":448,"Attempt":0,"Launch Time":1427397550489,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550496,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":314109,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3458,"Index":452,"Attempt":0,"Launch Time":1427397550497,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3397,"Index":391,"Attempt":0,"Launch Time":1427397550356,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550497,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":61,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":332815,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3459,"Index":453,"Attempt":0,"Launch Time":1427397550497,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3455,"Index":449,"Attempt":0,"Launch Time":1427397550490,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550497,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":349915,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3460,"Index":454,"Attempt":0,"Launch Time":1427397550502,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3456,"Index":450,"Attempt":0,"Launch Time":1427397550495,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550502,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":323477,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3461,"Index":455,"Attempt":0,"Launch Time":1427397550503,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3459,"Index":453,"Attempt":0,"Launch Time":1427397550497,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550504,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":270810,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3462,"Index":456,"Attempt":0,"Launch Time":1427397550505,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3457,"Index":451,"Attempt":0,"Launch Time":1427397550496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550505,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":339258,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3463,"Index":457,"Attempt":0,"Launch Time":1427397550509,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3460,"Index":454,"Attempt":0,"Launch Time":1427397550502,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550510,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":340127,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3464,"Index":458,"Attempt":0,"Launch Time":1427397550511,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3461,"Index":455,"Attempt":0,"Launch Time":1427397550503,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550511,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":321615,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3465,"Index":459,"Attempt":0,"Launch Time":1427397550511,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3458,"Index":452,"Attempt":0,"Launch Time":1427397550497,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550511,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":406534,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3466,"Index":460,"Attempt":0,"Launch Time":1427397550513,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3462,"Index":456,"Attempt":0,"Launch Time":1427397550505,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550513,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":326111,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3467,"Index":461,"Attempt":0,"Launch Time":1427397550525,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3464,"Index":458,"Attempt":0,"Launch Time":1427397550511,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550525,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":347458,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3468,"Index":462,"Attempt":0,"Launch Time":1427397550526,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3465,"Index":459,"Attempt":0,"Launch Time":1427397550511,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550526,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":362293,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3469,"Index":463,"Attempt":0,"Launch Time":1427397550528,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3466,"Index":460,"Attempt":0,"Launch Time":1427397550513,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550528,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":388619,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3470,"Index":464,"Attempt":0,"Launch Time":1427397550532,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3448,"Index":442,"Attempt":0,"Launch Time":1427397550478,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550532,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":52,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":382824,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3471,"Index":465,"Attempt":0,"Launch Time":1427397550532,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3467,"Index":461,"Attempt":0,"Launch Time":1427397550525,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550533,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":366910,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3472,"Index":466,"Attempt":0,"Launch Time":1427397550535,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3468,"Index":462,"Attempt":0,"Launch Time":1427397550526,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550535,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":344424,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3473,"Index":467,"Attempt":0,"Launch Time":1427397550536,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3469,"Index":463,"Attempt":0,"Launch Time":1427397550528,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550536,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9607,"Shuffle Write Time":299905,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3474,"Index":468,"Attempt":0,"Launch Time":1427397550541,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3471,"Index":465,"Attempt":0,"Launch Time":1427397550532,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550541,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":379981,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3475,"Index":469,"Attempt":0,"Launch Time":1427397550543,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3470,"Index":464,"Attempt":0,"Launch Time":1427397550532,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550543,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":346297,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3476,"Index":470,"Attempt":0,"Launch Time":1427397550543,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3472,"Index":466,"Attempt":0,"Launch Time":1427397550535,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550543,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":561749,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3477,"Index":471,"Attempt":0,"Launch Time":1427397550545,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3473,"Index":467,"Attempt":0,"Launch Time":1427397550536,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550545,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":409328,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3478,"Index":472,"Attempt":0,"Launch Time":1427397550550,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3474,"Index":468,"Attempt":0,"Launch Time":1427397550541,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550550,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":306166,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3479,"Index":473,"Attempt":0,"Launch Time":1427397550552,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3475,"Index":469,"Attempt":0,"Launch Time":1427397550543,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550552,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":304543,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3480,"Index":474,"Attempt":0,"Launch Time":1427397550554,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3477,"Index":471,"Attempt":0,"Launch Time":1427397550545,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550554,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9613,"Shuffle Write Time":305615,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3481,"Index":475,"Attempt":0,"Launch Time":1427397550557,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3478,"Index":472,"Attempt":0,"Launch Time":1427397550550,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550558,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":405334,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3482,"Index":476,"Attempt":0,"Launch Time":1427397550559,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3479,"Index":473,"Attempt":0,"Launch Time":1427397550552,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550559,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":290432,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3483,"Index":477,"Attempt":0,"Launch Time":1427397550560,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3476,"Index":470,"Attempt":0,"Launch Time":1427397550543,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550560,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":324060,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3484,"Index":478,"Attempt":0,"Launch Time":1427397550561,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3480,"Index":474,"Attempt":0,"Launch Time":1427397550554,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550561,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":290190,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3485,"Index":479,"Attempt":0,"Launch Time":1427397550565,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3481,"Index":475,"Attempt":0,"Launch Time":1427397550557,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550565,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":311588,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3486,"Index":480,"Attempt":0,"Launch Time":1427397550567,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3482,"Index":476,"Attempt":0,"Launch Time":1427397550559,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550567,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":395186,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3487,"Index":481,"Attempt":0,"Launch Time":1427397550569,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3484,"Index":478,"Attempt":0,"Launch Time":1427397550561,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550569,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":381964,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3488,"Index":482,"Attempt":0,"Launch Time":1427397550571,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3452,"Index":446,"Attempt":0,"Launch Time":1427397550483,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550571,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":347897,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3489,"Index":483,"Attempt":0,"Launch Time":1427397550574,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3485,"Index":479,"Attempt":0,"Launch Time":1427397550565,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550574,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":363386,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3490,"Index":484,"Attempt":0,"Launch Time":1427397550576,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3491,"Index":485,"Attempt":0,"Launch Time":1427397550576,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3483,"Index":477,"Attempt":0,"Launch Time":1427397550560,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550576,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1051371,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3486,"Index":480,"Attempt":0,"Launch Time":1427397550567,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550576,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":377867,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3492,"Index":486,"Attempt":0,"Launch Time":1427397550578,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3487,"Index":481,"Attempt":0,"Launch Time":1427397550569,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550578,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":444427,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3493,"Index":487,"Attempt":0,"Launch Time":1427397550578,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3463,"Index":457,"Attempt":0,"Launch Time":1427397550509,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550578,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":68,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":425290,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3494,"Index":488,"Attempt":0,"Launch Time":1427397550581,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3489,"Index":483,"Attempt":0,"Launch Time":1427397550574,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550581,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":328761,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3495,"Index":489,"Attempt":0,"Launch Time":1427397550584,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3491,"Index":485,"Attempt":0,"Launch Time":1427397550576,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550585,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":324874,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3496,"Index":490,"Attempt":0,"Launch Time":1427397550586,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3493,"Index":487,"Attempt":0,"Launch Time":1427397550578,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550586,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":333952,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3497,"Index":491,"Attempt":0,"Launch Time":1427397550586,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3492,"Index":486,"Attempt":0,"Launch Time":1427397550578,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550587,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":310686,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3498,"Index":492,"Attempt":0,"Launch Time":1427397550587,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3490,"Index":484,"Attempt":0,"Launch Time":1427397550576,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550587,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":318529,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3499,"Index":493,"Attempt":0,"Launch Time":1427397550588,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3449,"Index":443,"Attempt":0,"Launch Time":1427397550479,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550588,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":328949,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3500,"Index":494,"Attempt":0,"Launch Time":1427397550590,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3494,"Index":488,"Attempt":0,"Launch Time":1427397550581,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550590,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":304938,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3501,"Index":495,"Attempt":0,"Launch Time":1427397550592,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3495,"Index":489,"Attempt":0,"Launch Time":1427397550584,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550593,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":305016,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3502,"Index":496,"Attempt":0,"Launch Time":1427397550593,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3496,"Index":490,"Attempt":0,"Launch Time":1427397550586,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550593,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":337223,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3503,"Index":497,"Attempt":0,"Launch Time":1427397550594,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3497,"Index":491,"Attempt":0,"Launch Time":1427397550586,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550594,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":336397,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3504,"Index":498,"Attempt":0,"Launch Time":1427397550594,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3498,"Index":492,"Attempt":0,"Launch Time":1427397550587,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550595,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":312525,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3505,"Index":499,"Attempt":0,"Launch Time":1427397550597,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3500,"Index":494,"Attempt":0,"Launch Time":1427397550590,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550598,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":324745,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3506,"Index":500,"Attempt":0,"Launch Time":1427397550600,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3499,"Index":493,"Attempt":0,"Launch Time":1427397550588,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550600,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":317117,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3507,"Index":501,"Attempt":0,"Launch Time":1427397550601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3502,"Index":496,"Attempt":0,"Launch Time":1427397550593,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550601,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":381192,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3508,"Index":502,"Attempt":0,"Launch Time":1427397550602,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3503,"Index":497,"Attempt":0,"Launch Time":1427397550594,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550602,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":333836,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3509,"Index":503,"Attempt":0,"Launch Time":1427397550604,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3505,"Index":499,"Attempt":0,"Launch Time":1427397550597,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550604,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":323902,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3510,"Index":504,"Attempt":0,"Launch Time":1427397550609,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3501,"Index":495,"Attempt":0,"Launch Time":1427397550592,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550609,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":326240,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3511,"Index":505,"Attempt":0,"Launch Time":1427397550610,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3506,"Index":500,"Attempt":0,"Launch Time":1427397550600,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550610,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":2619212,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3512,"Index":506,"Attempt":0,"Launch Time":1427397550624,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3451,"Index":445,"Attempt":0,"Launch Time":1427397550481,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550625,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":14,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9605,"Shuffle Write Time":332404,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3513,"Index":507,"Attempt":0,"Launch Time":1427397550625,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3511,"Index":505,"Attempt":0,"Launch Time":1427397550610,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550625,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":325839,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3514,"Index":508,"Attempt":0,"Launch Time":1427397550627,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3488,"Index":482,"Attempt":0,"Launch Time":1427397550571,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550627,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":421653,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3515,"Index":509,"Attempt":0,"Launch Time":1427397550628,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3507,"Index":501,"Attempt":0,"Launch Time":1427397550601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550628,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":26,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":366611,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3516,"Index":510,"Attempt":0,"Launch Time":1427397550634,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3512,"Index":506,"Attempt":0,"Launch Time":1427397550624,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550634,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":393862,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3517,"Index":511,"Attempt":0,"Launch Time":1427397550636,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3518,"Index":512,"Attempt":0,"Launch Time":1427397550636,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3513,"Index":507,"Attempt":0,"Launch Time":1427397550625,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550636,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":430528,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3514,"Index":508,"Attempt":0,"Launch Time":1427397550627,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550636,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":350242,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3519,"Index":513,"Attempt":0,"Launch Time":1427397550637,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3508,"Index":502,"Attempt":0,"Launch Time":1427397550602,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550637,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":324040,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3520,"Index":514,"Attempt":0,"Launch Time":1427397550639,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3515,"Index":509,"Attempt":0,"Launch Time":1427397550628,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550639,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":393007,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3521,"Index":515,"Attempt":0,"Launch Time":1427397550643,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3517,"Index":511,"Attempt":0,"Launch Time":1427397550636,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550643,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":348944,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3522,"Index":516,"Attempt":0,"Launch Time":1427397550643,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3518,"Index":512,"Attempt":0,"Launch Time":1427397550636,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550644,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":363549,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3523,"Index":517,"Attempt":0,"Launch Time":1427397550644,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3516,"Index":510,"Attempt":0,"Launch Time":1427397550634,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550644,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":345980,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3524,"Index":518,"Attempt":0,"Launch Time":1427397550648,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3520,"Index":514,"Attempt":0,"Launch Time":1427397550639,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550648,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":330354,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3525,"Index":519,"Attempt":0,"Launch Time":1427397550652,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3523,"Index":517,"Attempt":0,"Launch Time":1427397550644,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550652,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":340929,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3526,"Index":520,"Attempt":0,"Launch Time":1427397550652,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3521,"Index":515,"Attempt":0,"Launch Time":1427397550643,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550653,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":325817,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3527,"Index":521,"Attempt":0,"Launch Time":1427397550659,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3522,"Index":516,"Attempt":0,"Launch Time":1427397550643,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550660,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":318223,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3528,"Index":522,"Attempt":0,"Launch Time":1427397550660,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3525,"Index":519,"Attempt":0,"Launch Time":1427397550652,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550660,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":371697,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3529,"Index":523,"Attempt":0,"Launch Time":1427397550660,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3526,"Index":520,"Attempt":0,"Launch Time":1427397550652,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550661,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":392070,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3530,"Index":524,"Attempt":0,"Launch Time":1427397550670,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3529,"Index":523,"Attempt":0,"Launch Time":1427397550660,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550670,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":400243,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3531,"Index":525,"Attempt":0,"Launch Time":1427397550670,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3528,"Index":522,"Attempt":0,"Launch Time":1427397550660,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550670,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":624123,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3532,"Index":526,"Attempt":0,"Launch Time":1427397550670,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3524,"Index":518,"Attempt":0,"Launch Time":1427397550648,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550670,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":669491,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3533,"Index":527,"Attempt":0,"Launch Time":1427397550671,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3527,"Index":521,"Attempt":0,"Launch Time":1427397550659,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550671,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":330376,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3534,"Index":528,"Attempt":0,"Launch Time":1427397550673,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3519,"Index":513,"Attempt":0,"Launch Time":1427397550637,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550673,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":32,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":333795,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3535,"Index":529,"Attempt":0,"Launch Time":1427397550677,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3530,"Index":524,"Attempt":0,"Launch Time":1427397550670,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550677,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":321220,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3536,"Index":530,"Attempt":0,"Launch Time":1427397550677,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3531,"Index":525,"Attempt":0,"Launch Time":1427397550670,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550677,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":320351,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3537,"Index":531,"Attempt":0,"Launch Time":1427397550678,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3532,"Index":526,"Attempt":0,"Launch Time":1427397550670,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550678,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":345020,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3538,"Index":532,"Attempt":0,"Launch Time":1427397550682,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3534,"Index":528,"Attempt":0,"Launch Time":1427397550673,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550682,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":315328,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3539,"Index":533,"Attempt":0,"Launch Time":1427397550683,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3504,"Index":498,"Attempt":0,"Launch Time":1427397550594,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550683,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":88,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":331034,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3540,"Index":534,"Attempt":0,"Launch Time":1427397550685,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3535,"Index":529,"Attempt":0,"Launch Time":1427397550677,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550685,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":339161,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3541,"Index":535,"Attempt":0,"Launch Time":1427397550688,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3542,"Index":536,"Attempt":0,"Launch Time":1427397550688,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3537,"Index":531,"Attempt":0,"Launch Time":1427397550678,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550688,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":364116,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3536,"Index":530,"Attempt":0,"Launch Time":1427397550677,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550688,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":729269,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3543,"Index":537,"Attempt":0,"Launch Time":1427397550694,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3540,"Index":534,"Attempt":0,"Launch Time":1427397550685,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550694,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":368745,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3544,"Index":538,"Attempt":0,"Launch Time":1427397550696,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3541,"Index":535,"Attempt":0,"Launch Time":1427397550688,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550696,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":327048,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3545,"Index":539,"Attempt":0,"Launch Time":1427397550696,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3542,"Index":536,"Attempt":0,"Launch Time":1427397550688,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550696,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":367106,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3546,"Index":540,"Attempt":0,"Launch Time":1427397550704,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3547,"Index":541,"Attempt":0,"Launch Time":1427397550705,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3544,"Index":538,"Attempt":0,"Launch Time":1427397550696,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550705,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":354029,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3545,"Index":539,"Attempt":0,"Launch Time":1427397550696,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550705,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":319906,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3548,"Index":542,"Attempt":0,"Launch Time":1427397550706,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3538,"Index":532,"Attempt":0,"Launch Time":1427397550682,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550706,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":24,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":723600,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3549,"Index":543,"Attempt":0,"Launch Time":1427397550709,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3550,"Index":544,"Attempt":0,"Launch Time":1427397550709,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3551,"Index":545,"Attempt":0,"Launch Time":1427397550709,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3510,"Index":504,"Attempt":0,"Launch Time":1427397550609,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550709,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":83,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":328793,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3533,"Index":527,"Attempt":0,"Launch Time":1427397550671,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550710,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":38,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":923303,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3543,"Index":537,"Attempt":0,"Launch Time":1427397550694,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550710,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":334192,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3552,"Index":546,"Attempt":0,"Launch Time":1427397550712,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3547,"Index":541,"Attempt":0,"Launch Time":1427397550705,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550712,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":319015,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3553,"Index":547,"Attempt":0,"Launch Time":1427397550714,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3546,"Index":540,"Attempt":0,"Launch Time":1427397550704,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550714,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":845136,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3554,"Index":548,"Attempt":0,"Launch Time":1427397550717,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3551,"Index":545,"Attempt":0,"Launch Time":1427397550709,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550718,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":356025,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3555,"Index":549,"Attempt":0,"Launch Time":1427397550718,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3549,"Index":543,"Attempt":0,"Launch Time":1427397550709,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550718,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":340801,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3556,"Index":550,"Attempt":0,"Launch Time":1427397550734,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3552,"Index":546,"Attempt":0,"Launch Time":1427397550712,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550735,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":328276,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3557,"Index":551,"Attempt":0,"Launch Time":1427397550735,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3548,"Index":542,"Attempt":0,"Launch Time":1427397550706,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550735,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":28,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1946356,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3558,"Index":552,"Attempt":0,"Launch Time":1427397550738,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3555,"Index":549,"Attempt":0,"Launch Time":1427397550718,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550738,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":18,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":341079,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3559,"Index":553,"Attempt":0,"Launch Time":1427397550739,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3550,"Index":544,"Attempt":0,"Launch Time":1427397550709,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550739,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":334345,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3560,"Index":554,"Attempt":0,"Launch Time":1427397550747,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3557,"Index":551,"Attempt":0,"Launch Time":1427397550735,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550747,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":339455,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3561,"Index":555,"Attempt":0,"Launch Time":1427397550748,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3558,"Index":552,"Attempt":0,"Launch Time":1427397550738,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550748,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":351987,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3559,"Index":553,"Attempt":0,"Launch Time":1427397550739,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550749,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":366164,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3562,"Index":556,"Attempt":0,"Launch Time":1427397550749,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3563,"Index":557,"Attempt":0,"Launch Time":1427397550754,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3539,"Index":533,"Attempt":0,"Launch Time":1427397550683,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550754,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":342873,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3564,"Index":558,"Attempt":0,"Launch Time":1427397550754,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3560,"Index":554,"Attempt":0,"Launch Time":1427397550747,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550754,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":350622,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3565,"Index":559,"Attempt":0,"Launch Time":1427397550755,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3561,"Index":555,"Attempt":0,"Launch Time":1427397550748,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550755,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":340525,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3566,"Index":560,"Attempt":0,"Launch Time":1427397550761,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3509,"Index":503,"Attempt":0,"Launch Time":1427397550604,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550761,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":143,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":357352,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3567,"Index":561,"Attempt":0,"Launch Time":1427397550762,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3563,"Index":557,"Attempt":0,"Launch Time":1427397550754,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550762,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":362684,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3568,"Index":562,"Attempt":0,"Launch Time":1427397550763,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3565,"Index":559,"Attempt":0,"Launch Time":1427397550755,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550763,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":451346,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3569,"Index":563,"Attempt":0,"Launch Time":1427397550766,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3562,"Index":556,"Attempt":0,"Launch Time":1427397550749,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550767,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":330893,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3570,"Index":564,"Attempt":0,"Launch Time":1427397550769,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3567,"Index":561,"Attempt":0,"Launch Time":1427397550762,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550769,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":360945,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3571,"Index":565,"Attempt":0,"Launch Time":1427397550771,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3568,"Index":562,"Attempt":0,"Launch Time":1427397550763,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550771,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":396378,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3572,"Index":566,"Attempt":0,"Launch Time":1427397550773,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3564,"Index":558,"Attempt":0,"Launch Time":1427397550754,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550773,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":419185,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3573,"Index":567,"Attempt":0,"Launch Time":1427397550777,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3570,"Index":564,"Attempt":0,"Launch Time":1427397550769,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550777,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":370212,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3574,"Index":568,"Attempt":0,"Launch Time":1427397550778,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3571,"Index":565,"Attempt":0,"Launch Time":1427397550771,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550778,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":324474,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3575,"Index":569,"Attempt":0,"Launch Time":1427397550782,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3566,"Index":560,"Attempt":0,"Launch Time":1427397550761,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550782,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":399018,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3576,"Index":570,"Attempt":0,"Launch Time":1427397550787,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3577,"Index":571,"Attempt":0,"Launch Time":1427397550787,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3574,"Index":568,"Attempt":0,"Launch Time":1427397550778,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550787,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":338337,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3572,"Index":566,"Attempt":0,"Launch Time":1427397550773,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550787,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":353628,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3578,"Index":572,"Attempt":0,"Launch Time":1427397550788,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3573,"Index":567,"Attempt":0,"Launch Time":1427397550777,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550788,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":335726,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3579,"Index":573,"Attempt":0,"Launch Time":1427397550792,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3569,"Index":563,"Attempt":0,"Launch Time":1427397550766,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550792,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":19,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":317492,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3580,"Index":574,"Attempt":0,"Launch Time":1427397550795,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3581,"Index":575,"Attempt":0,"Launch Time":1427397550796,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3577,"Index":571,"Attempt":0,"Launch Time":1427397550787,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550796,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":455439,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3576,"Index":570,"Attempt":0,"Launch Time":1427397550787,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550796,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":373344,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3582,"Index":576,"Attempt":0,"Launch Time":1427397550796,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3575,"Index":569,"Attempt":0,"Launch Time":1427397550782,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550797,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":6,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":383355,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3583,"Index":577,"Attempt":0,"Launch Time":1427397550801,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3578,"Index":572,"Attempt":0,"Launch Time":1427397550788,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550801,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9614,"Shuffle Write Time":322642,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3579,"Index":573,"Attempt":0,"Launch Time":1427397550792,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550801,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":328898,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3584,"Index":578,"Attempt":0,"Launch Time":1427397550801,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3585,"Index":579,"Attempt":0,"Launch Time":1427397550803,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3581,"Index":575,"Attempt":0,"Launch Time":1427397550796,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550803,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":339956,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3586,"Index":580,"Attempt":0,"Launch Time":1427397550805,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3580,"Index":574,"Attempt":0,"Launch Time":1427397550795,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550805,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":338775,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3587,"Index":581,"Attempt":0,"Launch Time":1427397550808,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3582,"Index":576,"Attempt":0,"Launch Time":1427397550796,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550808,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":308529,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3588,"Index":582,"Attempt":0,"Launch Time":1427397550810,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3585,"Index":579,"Attempt":0,"Launch Time":1427397550803,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550810,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":351263,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3589,"Index":583,"Attempt":0,"Launch Time":1427397550813,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3584,"Index":578,"Attempt":0,"Launch Time":1427397550801,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550813,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":365657,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3590,"Index":584,"Attempt":0,"Launch Time":1427397550815,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3586,"Index":580,"Attempt":0,"Launch Time":1427397550805,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550815,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9611,"Shuffle Write Time":406400,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3591,"Index":585,"Attempt":0,"Launch Time":1427397550818,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3588,"Index":582,"Attempt":0,"Launch Time":1427397550810,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550818,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":354655,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3592,"Index":586,"Attempt":0,"Launch Time":1427397550819,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3583,"Index":577,"Attempt":0,"Launch Time":1427397550801,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550819,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1105697,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3593,"Index":587,"Attempt":0,"Launch Time":1427397550822,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3589,"Index":583,"Attempt":0,"Launch Time":1427397550813,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550822,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":304100,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3594,"Index":588,"Attempt":0,"Launch Time":1427397550824,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3587,"Index":581,"Attempt":0,"Launch Time":1427397550808,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550824,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":589127,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3595,"Index":589,"Attempt":0,"Launch Time":1427397550825,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3590,"Index":584,"Attempt":0,"Launch Time":1427397550815,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550825,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":338846,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3596,"Index":590,"Attempt":0,"Launch Time":1427397550825,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3591,"Index":585,"Attempt":0,"Launch Time":1427397550818,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550825,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":333207,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3597,"Index":591,"Attempt":0,"Launch Time":1427397550826,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3592,"Index":586,"Attempt":0,"Launch Time":1427397550819,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550827,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":315836,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3598,"Index":592,"Attempt":0,"Launch Time":1427397550830,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3593,"Index":587,"Attempt":0,"Launch Time":1427397550822,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550831,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":348872,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3599,"Index":593,"Attempt":0,"Launch Time":1427397550833,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3595,"Index":589,"Attempt":0,"Launch Time":1427397550825,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550833,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":395875,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3600,"Index":594,"Attempt":0,"Launch Time":1427397550833,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3596,"Index":590,"Attempt":0,"Launch Time":1427397550825,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550834,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":452620,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3601,"Index":595,"Attempt":0,"Launch Time":1427397550842,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3553,"Index":547,"Attempt":0,"Launch Time":1427397550714,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550843,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":127,"Result Size":930,"JVM GC Time":15,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":432694,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3602,"Index":596,"Attempt":0,"Launch Time":1427397550846,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3598,"Index":592,"Attempt":0,"Launch Time":1427397550830,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550846,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":320741,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3603,"Index":597,"Attempt":0,"Launch Time":1427397550846,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3600,"Index":594,"Attempt":0,"Launch Time":1427397550833,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550846,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":333569,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3604,"Index":598,"Attempt":0,"Launch Time":1427397550847,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3597,"Index":591,"Attempt":0,"Launch Time":1427397550826,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550847,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":20,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":335591,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3605,"Index":599,"Attempt":0,"Launch Time":1427397550853,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3594,"Index":588,"Attempt":0,"Launch Time":1427397550824,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550853,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":27,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9608,"Shuffle Write Time":617334,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3606,"Index":600,"Attempt":0,"Launch Time":1427397550853,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3602,"Index":596,"Attempt":0,"Launch Time":1427397550846,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550854,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":323574,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3607,"Index":601,"Attempt":0,"Launch Time":1427397550854,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3603,"Index":597,"Attempt":0,"Launch Time":1427397550846,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550854,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":310985,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3608,"Index":602,"Attempt":0,"Launch Time":1427397550855,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3604,"Index":598,"Attempt":0,"Launch Time":1427397550847,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550855,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":441448,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3609,"Index":603,"Attempt":0,"Launch Time":1427397550856,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3599,"Index":593,"Attempt":0,"Launch Time":1427397550833,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550856,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":21,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":344369,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3610,"Index":604,"Attempt":0,"Launch Time":1427397550859,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3554,"Index":548,"Attempt":0,"Launch Time":1427397550717,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550859,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":140,"Result Size":930,"JVM GC Time":15,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":110325665,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3611,"Index":605,"Attempt":0,"Launch Time":1427397550861,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3556,"Index":550,"Attempt":0,"Launch Time":1427397550734,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550862,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":355778,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3612,"Index":606,"Attempt":0,"Launch Time":1427397550863,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3601,"Index":595,"Attempt":0,"Launch Time":1427397550842,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550863,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":330264,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3613,"Index":607,"Attempt":0,"Launch Time":1427397550867,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3610,"Index":604,"Attempt":0,"Launch Time":1427397550859,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550867,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":392555,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3614,"Index":608,"Attempt":0,"Launch Time":1427397550869,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3611,"Index":605,"Attempt":0,"Launch Time":1427397550861,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550869,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":354673,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3615,"Index":609,"Attempt":0,"Launch Time":1427397550871,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3612,"Index":606,"Attempt":0,"Launch Time":1427397550863,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550871,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":324187,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3616,"Index":610,"Attempt":0,"Launch Time":1427397550877,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3613,"Index":607,"Attempt":0,"Launch Time":1427397550867,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550877,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":331741,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3617,"Index":611,"Attempt":0,"Launch Time":1427397550878,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3607,"Index":601,"Attempt":0,"Launch Time":1427397550854,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550878,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":24,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":374820,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3618,"Index":612,"Attempt":0,"Launch Time":1427397550879,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3614,"Index":608,"Attempt":0,"Launch Time":1427397550869,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550879,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1915385,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3619,"Index":613,"Attempt":0,"Launch Time":1427397550879,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3615,"Index":609,"Attempt":0,"Launch Time":1427397550871,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550880,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":394987,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3620,"Index":614,"Attempt":0,"Launch Time":1427397550885,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3616,"Index":610,"Attempt":0,"Launch Time":1427397550877,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550885,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":318660,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3621,"Index":615,"Attempt":0,"Launch Time":1427397550888,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3622,"Index":616,"Attempt":0,"Launch Time":1427397550889,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3618,"Index":612,"Attempt":0,"Launch Time":1427397550879,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550889,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":376714,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3605,"Index":599,"Attempt":0,"Launch Time":1427397550853,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550889,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":34,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":378009,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3623,"Index":617,"Attempt":0,"Launch Time":1427397550889,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3619,"Index":613,"Attempt":0,"Launch Time":1427397550879,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550889,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":357994,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3624,"Index":618,"Attempt":0,"Launch Time":1427397550894,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3625,"Index":619,"Attempt":0,"Launch Time":1427397550894,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3620,"Index":614,"Attempt":0,"Launch Time":1427397550885,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550894,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":294489,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3617,"Index":611,"Attempt":0,"Launch Time":1427397550878,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550894,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":319148,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3626,"Index":620,"Attempt":0,"Launch Time":1427397550897,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3627,"Index":621,"Attempt":0,"Launch Time":1427397550897,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3622,"Index":616,"Attempt":0,"Launch Time":1427397550889,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550897,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":313032,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3621,"Index":615,"Attempt":0,"Launch Time":1427397550888,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550897,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":327866,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3628,"Index":622,"Attempt":0,"Launch Time":1427397550897,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3623,"Index":617,"Attempt":0,"Launch Time":1427397550889,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550897,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":279345,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3629,"Index":623,"Attempt":0,"Launch Time":1427397550901,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3625,"Index":619,"Attempt":0,"Launch Time":1427397550894,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550901,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":349175,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3630,"Index":624,"Attempt":0,"Launch Time":1427397550903,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3624,"Index":618,"Attempt":0,"Launch Time":1427397550894,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550903,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":417954,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3631,"Index":625,"Attempt":0,"Launch Time":1427397550906,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3627,"Index":621,"Attempt":0,"Launch Time":1427397550897,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550906,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":341997,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3632,"Index":626,"Attempt":0,"Launch Time":1427397550909,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3629,"Index":623,"Attempt":0,"Launch Time":1427397550901,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550909,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":335518,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3633,"Index":627,"Attempt":0,"Launch Time":1427397550910,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3626,"Index":620,"Attempt":0,"Launch Time":1427397550897,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550910,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":360712,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3634,"Index":628,"Attempt":0,"Launch Time":1427397550911,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3630,"Index":624,"Attempt":0,"Launch Time":1427397550903,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550911,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":379661,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3635,"Index":629,"Attempt":0,"Launch Time":1427397550914,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3631,"Index":625,"Attempt":0,"Launch Time":1427397550906,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550914,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":400091,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3636,"Index":630,"Attempt":0,"Launch Time":1427397550917,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3628,"Index":622,"Attempt":0,"Launch Time":1427397550897,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550917,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":441849,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3637,"Index":631,"Attempt":0,"Launch Time":1427397550918,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3632,"Index":626,"Attempt":0,"Launch Time":1427397550909,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550918,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":455218,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3638,"Index":632,"Attempt":0,"Launch Time":1427397550918,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3608,"Index":602,"Attempt":0,"Launch Time":1427397550855,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550918,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":39,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":381036,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3639,"Index":633,"Attempt":0,"Launch Time":1427397550920,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3634,"Index":628,"Attempt":0,"Launch Time":1427397550911,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550920,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":434108,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3640,"Index":634,"Attempt":0,"Launch Time":1427397550920,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3633,"Index":627,"Attempt":0,"Launch Time":1427397550910,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550920,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":523592,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3641,"Index":635,"Attempt":0,"Launch Time":1427397550923,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3635,"Index":629,"Attempt":0,"Launch Time":1427397550914,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550923,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":327563,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3642,"Index":636,"Attempt":0,"Launch Time":1427397550925,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3637,"Index":631,"Attempt":0,"Launch Time":1427397550918,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550925,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":403142,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3643,"Index":637,"Attempt":0,"Launch Time":1427397550927,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3639,"Index":633,"Attempt":0,"Launch Time":1427397550920,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550927,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":323447,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3644,"Index":638,"Attempt":0,"Launch Time":1427397550928,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3640,"Index":634,"Attempt":0,"Launch Time":1427397550920,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550928,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":332006,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3645,"Index":639,"Attempt":0,"Launch Time":1427397550931,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3641,"Index":635,"Attempt":0,"Launch Time":1427397550923,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550937,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":299097,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3646,"Index":640,"Attempt":0,"Launch Time":1427397550940,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3643,"Index":637,"Attempt":0,"Launch Time":1427397550927,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550940,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":433144,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3647,"Index":641,"Attempt":0,"Launch Time":1427397550944,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3642,"Index":636,"Attempt":0,"Launch Time":1427397550925,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550944,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":19,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":11277309,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3648,"Index":642,"Attempt":0,"Launch Time":1427397550945,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3645,"Index":639,"Attempt":0,"Launch Time":1427397550931,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550945,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":331458,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3649,"Index":643,"Attempt":0,"Launch Time":1427397550947,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3646,"Index":640,"Attempt":0,"Launch Time":1427397550940,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550947,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":302717,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3650,"Index":644,"Attempt":0,"Launch Time":1427397550949,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3644,"Index":638,"Attempt":0,"Launch Time":1427397550928,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550949,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":306280,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3651,"Index":645,"Attempt":0,"Launch Time":1427397550951,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3647,"Index":641,"Attempt":0,"Launch Time":1427397550944,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550951,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":341462,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3652,"Index":646,"Attempt":0,"Launch Time":1427397550955,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3636,"Index":630,"Attempt":0,"Launch Time":1427397550917,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550955,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":32,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":362593,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3653,"Index":647,"Attempt":0,"Launch Time":1427397550955,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3606,"Index":600,"Attempt":0,"Launch Time":1427397550853,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550955,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":101,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":360815,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3654,"Index":648,"Attempt":0,"Launch Time":1427397550955,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3649,"Index":643,"Attempt":0,"Launch Time":1427397550947,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550955,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":325407,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3655,"Index":649,"Attempt":0,"Launch Time":1427397550959,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3651,"Index":645,"Attempt":0,"Launch Time":1427397550951,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550959,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":361288,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3656,"Index":650,"Attempt":0,"Launch Time":1427397550960,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3648,"Index":642,"Attempt":0,"Launch Time":1427397550945,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550960,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":373623,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3657,"Index":651,"Attempt":0,"Launch Time":1427397550961,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3650,"Index":644,"Attempt":0,"Launch Time":1427397550949,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550961,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":371743,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3658,"Index":652,"Attempt":0,"Launch Time":1427397550963,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3653,"Index":647,"Attempt":0,"Launch Time":1427397550955,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550963,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":372464,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3659,"Index":653,"Attempt":0,"Launch Time":1427397550963,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3652,"Index":646,"Attempt":0,"Launch Time":1427397550955,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550963,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":331124,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3660,"Index":654,"Attempt":0,"Launch Time":1427397550969,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3654,"Index":648,"Attempt":0,"Launch Time":1427397550955,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550969,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":395470,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3661,"Index":655,"Attempt":0,"Launch Time":1427397550970,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3656,"Index":650,"Attempt":0,"Launch Time":1427397550960,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550970,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":313797,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3662,"Index":656,"Attempt":0,"Launch Time":1427397550970,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3655,"Index":649,"Attempt":0,"Launch Time":1427397550959,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550971,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":317085,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3663,"Index":657,"Attempt":0,"Launch Time":1427397550971,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3659,"Index":653,"Attempt":0,"Launch Time":1427397550963,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550971,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":344852,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3664,"Index":658,"Attempt":0,"Launch Time":1427397550975,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3658,"Index":652,"Attempt":0,"Launch Time":1427397550963,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550975,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":635443,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3665,"Index":659,"Attempt":0,"Launch Time":1427397550977,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3662,"Index":656,"Attempt":0,"Launch Time":1427397550970,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550978,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":327254,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3666,"Index":660,"Attempt":0,"Launch Time":1427397550978,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3663,"Index":657,"Attempt":0,"Launch Time":1427397550971,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550978,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":326103,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3667,"Index":661,"Attempt":0,"Launch Time":1427397550980,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3609,"Index":603,"Attempt":0,"Launch Time":1427397550856,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550980,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":94,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":327957,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3668,"Index":662,"Attempt":0,"Launch Time":1427397550982,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3638,"Index":632,"Attempt":0,"Launch Time":1427397550918,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550983,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":58,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":841939,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3669,"Index":663,"Attempt":0,"Launch Time":1427397550984,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3665,"Index":659,"Attempt":0,"Launch Time":1427397550977,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550984,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":305322,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3670,"Index":664,"Attempt":0,"Launch Time":1427397550985,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3660,"Index":654,"Attempt":0,"Launch Time":1427397550969,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550985,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":324106,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3671,"Index":665,"Attempt":0,"Launch Time":1427397550986,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3667,"Index":661,"Attempt":0,"Launch Time":1427397550980,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550987,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":303600,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3672,"Index":666,"Attempt":0,"Launch Time":1427397550989,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3666,"Index":660,"Attempt":0,"Launch Time":1427397550978,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550989,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":323594,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3673,"Index":667,"Attempt":0,"Launch Time":1427397550989,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3668,"Index":662,"Attempt":0,"Launch Time":1427397550982,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550990,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":313189,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3674,"Index":668,"Attempt":0,"Launch Time":1427397550992,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3669,"Index":663,"Attempt":0,"Launch Time":1427397550984,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550993,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":311584,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3675,"Index":669,"Attempt":0,"Launch Time":1427397550993,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3671,"Index":665,"Attempt":0,"Launch Time":1427397550986,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550993,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":285728,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3676,"Index":670,"Attempt":0,"Launch Time":1427397550996,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3673,"Index":667,"Attempt":0,"Launch Time":1427397550989,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550996,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":297854,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3677,"Index":671,"Attempt":0,"Launch Time":1427397550996,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3672,"Index":666,"Attempt":0,"Launch Time":1427397550989,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550997,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":328000,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3678,"Index":672,"Attempt":0,"Launch Time":1427397550999,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3675,"Index":669,"Attempt":0,"Launch Time":1427397550993,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551000,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":263372,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3679,"Index":673,"Attempt":0,"Launch Time":1427397551001,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3674,"Index":668,"Attempt":0,"Launch Time":1427397550992,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551002,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":331583,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3680,"Index":674,"Attempt":0,"Launch Time":1427397551003,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3676,"Index":670,"Attempt":0,"Launch Time":1427397550996,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551003,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":331373,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3681,"Index":675,"Attempt":0,"Launch Time":1427397551004,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3677,"Index":671,"Attempt":0,"Launch Time":1427397550996,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551004,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":358844,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3682,"Index":676,"Attempt":0,"Launch Time":1427397551006,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3678,"Index":672,"Attempt":0,"Launch Time":1427397550999,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551006,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":300788,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3683,"Index":677,"Attempt":0,"Launch Time":1427397551010,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3684,"Index":678,"Attempt":0,"Launch Time":1427397551010,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3680,"Index":674,"Attempt":0,"Launch Time":1427397551003,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551011,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":349227,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3679,"Index":673,"Attempt":0,"Launch Time":1427397551001,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551011,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":393602,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3685,"Index":679,"Attempt":0,"Launch Time":1427397551013,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3686,"Index":680,"Attempt":0,"Launch Time":1427397551013,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3682,"Index":676,"Attempt":0,"Launch Time":1427397551006,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551014,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":358627,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3681,"Index":675,"Attempt":0,"Launch Time":1427397551004,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551014,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1407392,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3687,"Index":681,"Attempt":0,"Launch Time":1427397551018,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3688,"Index":682,"Attempt":0,"Launch Time":1427397551018,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3664,"Index":658,"Attempt":0,"Launch Time":1427397550975,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551019,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":40,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":328489,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3683,"Index":677,"Attempt":0,"Launch Time":1427397551010,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551019,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":376943,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3689,"Index":683,"Attempt":0,"Launch Time":1427397551021,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3684,"Index":678,"Attempt":0,"Launch Time":1427397551010,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551021,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":306442,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3690,"Index":684,"Attempt":0,"Launch Time":1427397551027,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3685,"Index":679,"Attempt":0,"Launch Time":1427397551013,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551027,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":3,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":319840,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3691,"Index":685,"Attempt":0,"Launch Time":1427397551028,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3686,"Index":680,"Attempt":0,"Launch Time":1427397551013,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551028,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":3,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":350880,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3692,"Index":686,"Attempt":0,"Launch Time":1427397551029,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3670,"Index":664,"Attempt":0,"Launch Time":1427397550985,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551029,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":8,"Executor Run Time":17,"Result Size":930,"JVM GC Time":3,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":308194,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3693,"Index":687,"Attempt":0,"Launch Time":1427397551032,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3688,"Index":682,"Attempt":0,"Launch Time":1427397551018,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551032,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":6,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":286104,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3694,"Index":688,"Attempt":0,"Launch Time":1427397551034,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3690,"Index":684,"Attempt":0,"Launch Time":1427397551027,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551034,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":354697,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3695,"Index":689,"Attempt":0,"Launch Time":1427397551035,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3689,"Index":683,"Attempt":0,"Launch Time":1427397551021,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551035,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":341903,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3696,"Index":690,"Attempt":0,"Launch Time":1427397551035,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3691,"Index":685,"Attempt":0,"Launch Time":1427397551028,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551035,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":349609,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3697,"Index":691,"Attempt":0,"Launch Time":1427397551038,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3657,"Index":651,"Attempt":0,"Launch Time":1427397550961,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551038,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":75,"Result Size":930,"JVM GC Time":3,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":50093659,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3698,"Index":692,"Attempt":0,"Launch Time":1427397551041,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3692,"Index":686,"Attempt":0,"Launch Time":1427397551029,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551041,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":321257,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3699,"Index":693,"Attempt":0,"Launch Time":1427397551041,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3694,"Index":688,"Attempt":0,"Launch Time":1427397551034,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551041,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":333823,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3700,"Index":694,"Attempt":0,"Launch Time":1427397551042,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3696,"Index":690,"Attempt":0,"Launch Time":1427397551035,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551042,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":312412,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3701,"Index":695,"Attempt":0,"Launch Time":1427397551043,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3693,"Index":687,"Attempt":0,"Launch Time":1427397551032,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551043,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":305639,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3702,"Index":696,"Attempt":0,"Launch Time":1427397551045,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3697,"Index":691,"Attempt":0,"Launch Time":1427397551038,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551045,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":335432,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3703,"Index":697,"Attempt":0,"Launch Time":1427397551047,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3695,"Index":689,"Attempt":0,"Launch Time":1427397551035,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551047,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":311353,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3704,"Index":698,"Attempt":0,"Launch Time":1427397551048,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3698,"Index":692,"Attempt":0,"Launch Time":1427397551041,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551048,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":296672,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3705,"Index":699,"Attempt":0,"Launch Time":1427397551050,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3699,"Index":693,"Attempt":0,"Launch Time":1427397551041,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551051,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":328832,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3706,"Index":700,"Attempt":0,"Launch Time":1427397551053,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3702,"Index":696,"Attempt":0,"Launch Time":1427397551045,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551053,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":309215,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3707,"Index":701,"Attempt":0,"Launch Time":1427397551055,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3704,"Index":698,"Attempt":0,"Launch Time":1427397551048,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551055,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":315621,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3708,"Index":702,"Attempt":0,"Launch Time":1427397551056,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3703,"Index":697,"Attempt":0,"Launch Time":1427397551047,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551057,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9612,"Shuffle Write Time":308638,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3709,"Index":703,"Attempt":0,"Launch Time":1427397551058,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3710,"Index":704,"Attempt":0,"Launch Time":1427397551058,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3700,"Index":694,"Attempt":0,"Launch Time":1427397551042,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551058,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":317341,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3705,"Index":699,"Attempt":0,"Launch Time":1427397551050,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551058,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":358685,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3711,"Index":705,"Attempt":0,"Launch Time":1427397551061,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3706,"Index":700,"Attempt":0,"Launch Time":1427397551053,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551061,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":324579,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3712,"Index":706,"Attempt":0,"Launch Time":1427397551061,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3707,"Index":701,"Attempt":0,"Launch Time":1427397551055,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551062,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":322147,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3713,"Index":707,"Attempt":0,"Launch Time":1427397551064,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3714,"Index":708,"Attempt":0,"Launch Time":1427397551065,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3710,"Index":704,"Attempt":0,"Launch Time":1427397551058,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551065,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":306654,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3709,"Index":703,"Attempt":0,"Launch Time":1427397551058,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551065,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":287322,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3715,"Index":709,"Attempt":0,"Launch Time":1427397551066,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3701,"Index":695,"Attempt":0,"Launch Time":1427397551043,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551066,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":22,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":324557,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3716,"Index":710,"Attempt":0,"Launch Time":1427397551067,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3711,"Index":705,"Attempt":0,"Launch Time":1427397551061,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551068,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9615,"Shuffle Write Time":324246,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3717,"Index":711,"Attempt":0,"Launch Time":1427397551068,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3712,"Index":706,"Attempt":0,"Launch Time":1427397551061,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551068,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":301451,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3718,"Index":712,"Attempt":0,"Launch Time":1427397551071,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3719,"Index":713,"Attempt":0,"Launch Time":1427397551072,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3714,"Index":708,"Attempt":0,"Launch Time":1427397551065,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551072,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":331440,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3713,"Index":707,"Attempt":0,"Launch Time":1427397551064,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551072,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":342598,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3720,"Index":714,"Attempt":0,"Launch Time":1427397551073,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3715,"Index":709,"Attempt":0,"Launch Time":1427397551066,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551073,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":337154,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3721,"Index":715,"Attempt":0,"Launch Time":1427397551074,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3708,"Index":702,"Attempt":0,"Launch Time":1427397551056,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551074,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":386081,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3722,"Index":716,"Attempt":0,"Launch Time":1427397551075,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3723,"Index":717,"Attempt":0,"Launch Time":1427397551075,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3717,"Index":711,"Attempt":0,"Launch Time":1427397551068,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551075,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":290530,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3716,"Index":710,"Attempt":0,"Launch Time":1427397551067,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551075,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":319344,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3724,"Index":718,"Attempt":0,"Launch Time":1427397551078,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3725,"Index":719,"Attempt":0,"Launch Time":1427397551079,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3719,"Index":713,"Attempt":0,"Launch Time":1427397551072,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551079,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9613,"Shuffle Write Time":349748,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3718,"Index":712,"Attempt":0,"Launch Time":1427397551071,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551079,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":338939,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3726,"Index":720,"Attempt":0,"Launch Time":1427397551080,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3720,"Index":714,"Attempt":0,"Launch Time":1427397551073,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551080,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":338785,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3727,"Index":721,"Attempt":0,"Launch Time":1427397551082,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3722,"Index":716,"Attempt":0,"Launch Time":1427397551075,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551082,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":361805,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3728,"Index":722,"Attempt":0,"Launch Time":1427397551082,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3723,"Index":717,"Attempt":0,"Launch Time":1427397551075,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551082,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":407946,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3729,"Index":723,"Attempt":0,"Launch Time":1427397551086,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3730,"Index":724,"Attempt":0,"Launch Time":1427397551086,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3724,"Index":718,"Attempt":0,"Launch Time":1427397551078,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551086,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":380251,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3725,"Index":719,"Attempt":0,"Launch Time":1427397551079,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551086,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":324266,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3731,"Index":725,"Attempt":0,"Launch Time":1427397551087,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3726,"Index":720,"Attempt":0,"Launch Time":1427397551080,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551087,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":318314,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3732,"Index":726,"Attempt":0,"Launch Time":1427397551088,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3721,"Index":715,"Attempt":0,"Launch Time":1427397551074,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551088,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":606916,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3733,"Index":727,"Attempt":0,"Launch Time":1427397551089,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3727,"Index":721,"Attempt":0,"Launch Time":1427397551082,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551089,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":303503,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3734,"Index":728,"Attempt":0,"Launch Time":1427397551090,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3728,"Index":722,"Attempt":0,"Launch Time":1427397551082,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551090,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":311688,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3735,"Index":729,"Attempt":0,"Launch Time":1427397551099,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3736,"Index":730,"Attempt":0,"Launch Time":1427397551099,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3729,"Index":723,"Attempt":0,"Launch Time":1427397551086,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551099,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":324156,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3661,"Index":655,"Attempt":0,"Launch Time":1427397550970,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551099,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":120,"Result Size":930,"JVM GC Time":8,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":360928,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3737,"Index":731,"Attempt":0,"Launch Time":1427397551099,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3730,"Index":724,"Attempt":0,"Launch Time":1427397551086,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551099,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":360087,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3738,"Index":732,"Attempt":0,"Launch Time":1427397551101,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3732,"Index":726,"Attempt":0,"Launch Time":1427397551088,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551102,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":337339,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3739,"Index":733,"Attempt":0,"Launch Time":1427397551103,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3734,"Index":728,"Attempt":0,"Launch Time":1427397551090,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551103,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":307125,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3740,"Index":734,"Attempt":0,"Launch Time":1427397551105,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3735,"Index":729,"Attempt":0,"Launch Time":1427397551099,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551105,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":284903,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3741,"Index":735,"Attempt":0,"Launch Time":1427397551106,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3736,"Index":730,"Attempt":0,"Launch Time":1427397551099,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551106,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":323656,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3742,"Index":736,"Attempt":0,"Launch Time":1427397551108,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3738,"Index":732,"Attempt":0,"Launch Time":1427397551101,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551108,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":319112,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3743,"Index":737,"Attempt":0,"Launch Time":1427397551112,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3687,"Index":681,"Attempt":0,"Launch Time":1427397551018,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551112,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":82,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":311786,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3744,"Index":738,"Attempt":0,"Launch Time":1427397551119,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3743,"Index":737,"Attempt":0,"Launch Time":1427397551112,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551119,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":295464,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3745,"Index":739,"Attempt":0,"Launch Time":1427397551126,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3744,"Index":738,"Attempt":0,"Launch Time":1427397551119,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551126,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":294497,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3746,"Index":740,"Attempt":0,"Launch Time":1427397551128,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3741,"Index":735,"Attempt":0,"Launch Time":1427397551106,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551128,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":8,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":327599,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3747,"Index":741,"Attempt":0,"Launch Time":1427397551131,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3742,"Index":736,"Attempt":0,"Launch Time":1427397551108,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551132,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":324365,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3748,"Index":742,"Attempt":0,"Launch Time":1427397551135,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3737,"Index":731,"Attempt":0,"Launch Time":1427397551099,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551135,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":36,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":346677,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3749,"Index":743,"Attempt":0,"Launch Time":1427397551135,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3745,"Index":739,"Attempt":0,"Launch Time":1427397551126,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551136,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":333078,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3750,"Index":744,"Attempt":0,"Launch Time":1427397551136,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3746,"Index":740,"Attempt":0,"Launch Time":1427397551128,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551136,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":307185,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3751,"Index":745,"Attempt":0,"Launch Time":1427397551139,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3747,"Index":741,"Attempt":0,"Launch Time":1427397551131,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551139,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":352144,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3752,"Index":746,"Attempt":0,"Launch Time":1427397551143,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3749,"Index":743,"Attempt":0,"Launch Time":1427397551135,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551144,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":370422,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3753,"Index":747,"Attempt":0,"Launch Time":1427397551144,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3750,"Index":744,"Attempt":0,"Launch Time":1427397551136,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551144,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":307271,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3754,"Index":748,"Attempt":0,"Launch Time":1427397551147,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3751,"Index":745,"Attempt":0,"Launch Time":1427397551139,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551147,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":343442,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3755,"Index":749,"Attempt":0,"Launch Time":1427397551153,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3753,"Index":747,"Attempt":0,"Launch Time":1427397551144,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551153,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":342903,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3756,"Index":750,"Attempt":0,"Launch Time":1427397551155,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3748,"Index":742,"Attempt":0,"Launch Time":1427397551135,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551155,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":345750,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3757,"Index":751,"Attempt":0,"Launch Time":1427397551156,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3752,"Index":746,"Attempt":0,"Launch Time":1427397551143,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551157,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":378697,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3758,"Index":752,"Attempt":0,"Launch Time":1427397551161,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3755,"Index":749,"Attempt":0,"Launch Time":1427397551153,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551161,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":358213,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3759,"Index":753,"Attempt":0,"Launch Time":1427397551162,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3756,"Index":750,"Attempt":0,"Launch Time":1427397551155,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551162,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":306887,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3760,"Index":754,"Attempt":0,"Launch Time":1427397551164,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3757,"Index":751,"Attempt":0,"Launch Time":1427397551156,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551164,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":330076,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3761,"Index":755,"Attempt":0,"Launch Time":1427397551169,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3754,"Index":748,"Attempt":0,"Launch Time":1427397551147,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551169,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":19,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":345264,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3762,"Index":756,"Attempt":0,"Launch Time":1427397551171,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3760,"Index":754,"Attempt":0,"Launch Time":1427397551164,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551171,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":393299,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3763,"Index":757,"Attempt":0,"Launch Time":1427397551172,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3758,"Index":752,"Attempt":0,"Launch Time":1427397551161,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551172,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":387134,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3764,"Index":758,"Attempt":0,"Launch Time":1427397551175,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3731,"Index":725,"Attempt":0,"Launch Time":1427397551087,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551175,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":87,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":474164,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3765,"Index":759,"Attempt":0,"Launch Time":1427397551179,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3762,"Index":756,"Attempt":0,"Launch Time":1427397551171,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551179,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":390028,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3766,"Index":760,"Attempt":0,"Launch Time":1427397551179,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3761,"Index":755,"Attempt":0,"Launch Time":1427397551169,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551179,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":409115,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3767,"Index":761,"Attempt":0,"Launch Time":1427397551180,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3763,"Index":757,"Attempt":0,"Launch Time":1427397551172,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551180,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":406427,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3768,"Index":762,"Attempt":0,"Launch Time":1427397551182,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3759,"Index":753,"Attempt":0,"Launch Time":1427397551162,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551182,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":20,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":539119,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3769,"Index":763,"Attempt":0,"Launch Time":1427397551183,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3764,"Index":758,"Attempt":0,"Launch Time":1427397551175,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551183,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":360343,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3770,"Index":764,"Attempt":0,"Launch Time":1427397551186,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3766,"Index":760,"Attempt":0,"Launch Time":1427397551179,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551186,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":309213,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3771,"Index":765,"Attempt":0,"Launch Time":1427397551187,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3767,"Index":761,"Attempt":0,"Launch Time":1427397551180,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551187,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":338540,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3772,"Index":766,"Attempt":0,"Launch Time":1427397551188,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3765,"Index":759,"Attempt":0,"Launch Time":1427397551179,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551188,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":331154,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3773,"Index":767,"Attempt":0,"Launch Time":1427397551189,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3768,"Index":762,"Attempt":0,"Launch Time":1427397551182,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551189,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":290785,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3774,"Index":768,"Attempt":0,"Launch Time":1427397551193,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3770,"Index":764,"Attempt":0,"Launch Time":1427397551186,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551193,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":322991,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3775,"Index":769,"Attempt":0,"Launch Time":1427397551194,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3771,"Index":765,"Attempt":0,"Launch Time":1427397551187,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551194,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":301490,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3776,"Index":770,"Attempt":0,"Launch Time":1427397551195,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3769,"Index":763,"Attempt":0,"Launch Time":1427397551183,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551195,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":324259,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3777,"Index":771,"Attempt":0,"Launch Time":1427397551196,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3773,"Index":767,"Attempt":0,"Launch Time":1427397551189,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551196,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":328332,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3778,"Index":772,"Attempt":0,"Launch Time":1427397551196,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3772,"Index":766,"Attempt":0,"Launch Time":1427397551188,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551197,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":325763,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3779,"Index":773,"Attempt":0,"Launch Time":1427397551199,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3740,"Index":734,"Attempt":0,"Launch Time":1427397551105,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551199,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":6,"Executor Run Time":53,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":388269,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3780,"Index":774,"Attempt":0,"Launch Time":1427397551208,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3775,"Index":769,"Attempt":0,"Launch Time":1427397551194,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551208,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":442769,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3781,"Index":775,"Attempt":0,"Launch Time":1427397551208,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3774,"Index":768,"Attempt":0,"Launch Time":1427397551193,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551208,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":422319,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3782,"Index":776,"Attempt":0,"Launch Time":1427397551212,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3778,"Index":772,"Attempt":0,"Launch Time":1427397551196,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551212,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":340328,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3783,"Index":777,"Attempt":0,"Launch Time":1427397551213,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3779,"Index":773,"Attempt":0,"Launch Time":1427397551199,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551213,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":333833,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3784,"Index":778,"Attempt":0,"Launch Time":1427397551215,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3781,"Index":775,"Attempt":0,"Launch Time":1427397551208,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551215,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":329555,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3785,"Index":779,"Attempt":0,"Launch Time":1427397551216,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3780,"Index":774,"Attempt":0,"Launch Time":1427397551208,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551216,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":318427,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3786,"Index":780,"Attempt":0,"Launch Time":1427397551220,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3782,"Index":776,"Attempt":0,"Launch Time":1427397551212,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551220,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":334965,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3787,"Index":781,"Attempt":0,"Launch Time":1427397551220,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3783,"Index":777,"Attempt":0,"Launch Time":1427397551213,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551220,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":310302,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3788,"Index":782,"Attempt":0,"Launch Time":1427397551222,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3784,"Index":778,"Attempt":0,"Launch Time":1427397551215,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551222,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":348744,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3789,"Index":783,"Attempt":0,"Launch Time":1427397551225,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3777,"Index":771,"Attempt":0,"Launch Time":1427397551196,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551225,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":29,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":338062,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3790,"Index":784,"Attempt":0,"Launch Time":1427397551227,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3791,"Index":785,"Attempt":0,"Launch Time":1427397551227,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3786,"Index":780,"Attempt":0,"Launch Time":1427397551220,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551227,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":352031,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3785,"Index":779,"Attempt":0,"Launch Time":1427397551216,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551229,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":362552,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3792,"Index":786,"Attempt":0,"Launch Time":1427397551232,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3787,"Index":781,"Attempt":0,"Launch Time":1427397551220,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551232,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":368811,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3793,"Index":787,"Attempt":0,"Launch Time":1427397551232,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3776,"Index":770,"Attempt":0,"Launch Time":1427397551195,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551232,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":36,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":410900,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3794,"Index":788,"Attempt":0,"Launch Time":1427397551234,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3733,"Index":727,"Attempt":0,"Launch Time":1427397551089,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551234,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":145,"Result Size":930,"JVM GC Time":11,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":375105,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3795,"Index":789,"Attempt":0,"Launch Time":1427397551234,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3739,"Index":733,"Attempt":0,"Launch Time":1427397551103,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551234,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":131,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":441092,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3796,"Index":790,"Attempt":0,"Launch Time":1427397551236,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3791,"Index":785,"Attempt":0,"Launch Time":1427397551227,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551236,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":384088,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3797,"Index":791,"Attempt":0,"Launch Time":1427397551241,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3794,"Index":788,"Attempt":0,"Launch Time":1427397551234,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551241,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":344354,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3798,"Index":792,"Attempt":0,"Launch Time":1427397551243,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3795,"Index":789,"Attempt":0,"Launch Time":1427397551234,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551243,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":363799,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3799,"Index":793,"Attempt":0,"Launch Time":1427397551244,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3796,"Index":790,"Attempt":0,"Launch Time":1427397551236,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551244,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":354518,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3800,"Index":794,"Attempt":0,"Launch Time":1427397551248,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3797,"Index":791,"Attempt":0,"Launch Time":1427397551241,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551248,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":348194,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3801,"Index":795,"Attempt":0,"Launch Time":1427397551252,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3799,"Index":793,"Attempt":0,"Launch Time":1427397551244,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551252,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":345955,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3802,"Index":796,"Attempt":0,"Launch Time":1427397551255,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3800,"Index":794,"Attempt":0,"Launch Time":1427397551248,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551255,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":324721,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3803,"Index":797,"Attempt":0,"Launch Time":1427397551258,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3792,"Index":786,"Attempt":0,"Launch Time":1427397551232,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551258,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":2142651,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3804,"Index":798,"Attempt":0,"Launch Time":1427397551260,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3801,"Index":795,"Attempt":0,"Launch Time":1427397551252,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551260,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":359183,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3805,"Index":799,"Attempt":0,"Launch Time":1427397551263,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3802,"Index":796,"Attempt":0,"Launch Time":1427397551255,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551263,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":279528,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3806,"Index":800,"Attempt":0,"Launch Time":1427397551267,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3804,"Index":798,"Attempt":0,"Launch Time":1427397551260,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551267,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":346243,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3807,"Index":801,"Attempt":0,"Launch Time":1427397551271,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3805,"Index":799,"Attempt":0,"Launch Time":1427397551263,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551271,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":406899,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3808,"Index":802,"Attempt":0,"Launch Time":1427397551275,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3806,"Index":800,"Attempt":0,"Launch Time":1427397551267,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551275,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":383383,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3809,"Index":803,"Attempt":0,"Launch Time":1427397551278,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3803,"Index":797,"Attempt":0,"Launch Time":1427397551258,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551279,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":398112,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3810,"Index":804,"Attempt":0,"Launch Time":1427397551282,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3807,"Index":801,"Attempt":0,"Launch Time":1427397551271,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551282,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":788941,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3811,"Index":805,"Attempt":0,"Launch Time":1427397551284,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3808,"Index":802,"Attempt":0,"Launch Time":1427397551275,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551284,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":401511,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3812,"Index":806,"Attempt":0,"Launch Time":1427397551286,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3809,"Index":803,"Attempt":0,"Launch Time":1427397551278,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551286,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":281785,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3813,"Index":807,"Attempt":0,"Launch Time":1427397551288,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3789,"Index":783,"Attempt":0,"Launch Time":1427397551225,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551288,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":63,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1963714,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3814,"Index":808,"Attempt":0,"Launch Time":1427397551291,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3810,"Index":804,"Attempt":0,"Launch Time":1427397551282,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551291,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":318798,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3815,"Index":809,"Attempt":0,"Launch Time":1427397551295,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3812,"Index":806,"Attempt":0,"Launch Time":1427397551286,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551295,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1302904,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3816,"Index":810,"Attempt":0,"Launch Time":1427397551298,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3793,"Index":787,"Attempt":0,"Launch Time":1427397551232,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551299,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1693262,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3817,"Index":811,"Attempt":0,"Launch Time":1427397551300,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3814,"Index":808,"Attempt":0,"Launch Time":1427397551291,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551300,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":338570,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3818,"Index":812,"Attempt":0,"Launch Time":1427397551304,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3813,"Index":807,"Attempt":0,"Launch Time":1427397551288,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551304,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":918032,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3819,"Index":813,"Attempt":0,"Launch Time":1427397551306,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3815,"Index":809,"Attempt":0,"Launch Time":1427397551295,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551306,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":535287,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3820,"Index":814,"Attempt":0,"Launch Time":1427397551307,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3817,"Index":811,"Attempt":0,"Launch Time":1427397551300,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551308,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":320425,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3821,"Index":815,"Attempt":0,"Launch Time":1427397551311,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3818,"Index":812,"Attempt":0,"Launch Time":1427397551304,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551311,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":312862,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3822,"Index":816,"Attempt":0,"Launch Time":1427397551312,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3819,"Index":813,"Attempt":0,"Launch Time":1427397551306,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551312,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":290276,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3823,"Index":817,"Attempt":0,"Launch Time":1427397551316,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3820,"Index":814,"Attempt":0,"Launch Time":1427397551307,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551316,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":282563,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3824,"Index":818,"Attempt":0,"Launch Time":1427397551318,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3821,"Index":815,"Attempt":0,"Launch Time":1427397551311,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551318,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":311572,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3825,"Index":819,"Attempt":0,"Launch Time":1427397551319,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3822,"Index":816,"Attempt":0,"Launch Time":1427397551312,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551319,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":322449,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3826,"Index":820,"Attempt":0,"Launch Time":1427397551322,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3823,"Index":817,"Attempt":0,"Launch Time":1427397551316,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551332,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":297151,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3827,"Index":821,"Attempt":0,"Launch Time":1427397551333,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3788,"Index":782,"Attempt":0,"Launch Time":1427397551222,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551333,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":109,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":400425,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3828,"Index":822,"Attempt":0,"Launch Time":1427397551335,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3816,"Index":810,"Attempt":0,"Launch Time":1427397551298,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551336,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":36,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":355488,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3829,"Index":823,"Attempt":0,"Launch Time":1427397551336,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3824,"Index":818,"Attempt":0,"Launch Time":1427397551318,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551336,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":343146,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3830,"Index":824,"Attempt":0,"Launch Time":1427397551337,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3825,"Index":819,"Attempt":0,"Launch Time":1427397551319,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551337,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":758265,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3831,"Index":825,"Attempt":0,"Launch Time":1427397551344,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3790,"Index":784,"Attempt":0,"Launch Time":1427397551227,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551344,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":115,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":302894,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3832,"Index":826,"Attempt":0,"Launch Time":1427397551345,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3829,"Index":823,"Attempt":0,"Launch Time":1427397551336,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551345,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":289902,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3833,"Index":827,"Attempt":0,"Launch Time":1427397551346,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3826,"Index":820,"Attempt":0,"Launch Time":1427397551322,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551346,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":334041,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3834,"Index":828,"Attempt":0,"Launch Time":1427397551350,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3811,"Index":805,"Attempt":0,"Launch Time":1427397551284,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551350,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":65,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":388119,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3835,"Index":829,"Attempt":0,"Launch Time":1427397551352,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3798,"Index":792,"Attempt":0,"Launch Time":1427397551243,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551352,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":107,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":342572,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3836,"Index":830,"Attempt":0,"Launch Time":1427397551353,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3832,"Index":826,"Attempt":0,"Launch Time":1427397551345,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551353,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":341089,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3837,"Index":831,"Attempt":0,"Launch Time":1427397551353,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3833,"Index":827,"Attempt":0,"Launch Time":1427397551346,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551353,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":315257,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3838,"Index":832,"Attempt":0,"Launch Time":1427397551358,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3834,"Index":828,"Attempt":0,"Launch Time":1427397551350,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551358,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":411820,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3839,"Index":833,"Attempt":0,"Launch Time":1427397551359,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3835,"Index":829,"Attempt":0,"Launch Time":1427397551352,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551359,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":323311,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3840,"Index":834,"Attempt":0,"Launch Time":1427397551360,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3837,"Index":831,"Attempt":0,"Launch Time":1427397551353,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551360,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":317128,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3841,"Index":835,"Attempt":0,"Launch Time":1427397551361,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3836,"Index":830,"Attempt":0,"Launch Time":1427397551353,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551361,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":387511,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3842,"Index":836,"Attempt":0,"Launch Time":1427397551363,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3828,"Index":822,"Attempt":0,"Launch Time":1427397551335,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551363,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":26,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":319257,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3843,"Index":837,"Attempt":0,"Launch Time":1427397551369,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3841,"Index":835,"Attempt":0,"Launch Time":1427397551361,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551369,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":328016,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3844,"Index":838,"Attempt":0,"Launch Time":1427397551370,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3842,"Index":836,"Attempt":0,"Launch Time":1427397551363,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551371,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":412650,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3845,"Index":839,"Attempt":0,"Launch Time":1427397551376,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3831,"Index":825,"Attempt":0,"Launch Time":1427397551344,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551376,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":343466,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3846,"Index":840,"Attempt":0,"Launch Time":1427397551377,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3843,"Index":837,"Attempt":0,"Launch Time":1427397551369,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551377,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":327924,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3847,"Index":841,"Attempt":0,"Launch Time":1427397551379,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3827,"Index":821,"Attempt":0,"Launch Time":1427397551333,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551379,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":40,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":439456,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3848,"Index":842,"Attempt":0,"Launch Time":1427397551380,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3844,"Index":838,"Attempt":0,"Launch Time":1427397551370,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551380,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":360337,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3849,"Index":843,"Attempt":0,"Launch Time":1427397551384,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3845,"Index":839,"Attempt":0,"Launch Time":1427397551376,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551384,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":352309,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3850,"Index":844,"Attempt":0,"Launch Time":1427397551387,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3846,"Index":840,"Attempt":0,"Launch Time":1427397551377,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551387,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":303896,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3851,"Index":845,"Attempt":0,"Launch Time":1427397551388,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3847,"Index":841,"Attempt":0,"Launch Time":1427397551379,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551388,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":336248,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3852,"Index":846,"Attempt":0,"Launch Time":1427397551388,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3848,"Index":842,"Attempt":0,"Launch Time":1427397551380,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551388,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":547859,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3853,"Index":847,"Attempt":0,"Launch Time":1427397551394,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3849,"Index":843,"Attempt":0,"Launch Time":1427397551384,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551394,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":368606,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3854,"Index":848,"Attempt":0,"Launch Time":1427397551396,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3851,"Index":845,"Attempt":0,"Launch Time":1427397551388,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551396,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":340284,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3855,"Index":849,"Attempt":0,"Launch Time":1427397551398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3850,"Index":844,"Attempt":0,"Launch Time":1427397551387,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551398,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":575956,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3856,"Index":850,"Attempt":0,"Launch Time":1427397551399,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3838,"Index":832,"Attempt":0,"Launch Time":1427397551358,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551399,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":29,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":340774,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3857,"Index":851,"Attempt":0,"Launch Time":1427397551401,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3853,"Index":847,"Attempt":0,"Launch Time":1427397551394,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551401,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":303679,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3858,"Index":852,"Attempt":0,"Launch Time":1427397551404,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3854,"Index":848,"Attempt":0,"Launch Time":1427397551396,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551404,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":373879,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3859,"Index":853,"Attempt":0,"Launch Time":1427397551406,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3855,"Index":849,"Attempt":0,"Launch Time":1427397551398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551406,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":360590,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3860,"Index":854,"Attempt":0,"Launch Time":1427397551407,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3856,"Index":850,"Attempt":0,"Launch Time":1427397551399,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551407,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":330366,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3861,"Index":855,"Attempt":0,"Launch Time":1427397551408,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3857,"Index":851,"Attempt":0,"Launch Time":1427397551401,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551408,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":331278,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3862,"Index":856,"Attempt":0,"Launch Time":1427397551410,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3852,"Index":846,"Attempt":0,"Launch Time":1427397551388,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551410,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":19,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":380673,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3863,"Index":857,"Attempt":0,"Launch Time":1427397551411,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3858,"Index":852,"Attempt":0,"Launch Time":1427397551404,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551411,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":337634,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3864,"Index":858,"Attempt":0,"Launch Time":1427397551413,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3860,"Index":854,"Attempt":0,"Launch Time":1427397551407,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551414,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":307005,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3865,"Index":859,"Attempt":0,"Launch Time":1427397551415,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3861,"Index":855,"Attempt":0,"Launch Time":1427397551408,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551415,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":312298,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3866,"Index":860,"Attempt":0,"Launch Time":1427397551418,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3863,"Index":857,"Attempt":0,"Launch Time":1427397551411,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551418,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":286037,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3867,"Index":861,"Attempt":0,"Launch Time":1427397551421,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3859,"Index":853,"Attempt":0,"Launch Time":1427397551406,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551422,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":366161,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3868,"Index":862,"Attempt":0,"Launch Time":1427397551423,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3869,"Index":863,"Attempt":0,"Launch Time":1427397551423,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3862,"Index":856,"Attempt":0,"Launch Time":1427397551410,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551423,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":595053,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3865,"Index":859,"Attempt":0,"Launch Time":1427397551415,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551424,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":372542,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3870,"Index":864,"Attempt":0,"Launch Time":1427397551424,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3864,"Index":858,"Attempt":0,"Launch Time":1427397551413,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551425,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":388546,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3871,"Index":865,"Attempt":0,"Launch Time":1427397551425,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3866,"Index":860,"Attempt":0,"Launch Time":1427397551418,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551425,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":351318,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3872,"Index":866,"Attempt":0,"Launch Time":1427397551437,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3868,"Index":862,"Attempt":0,"Launch Time":1427397551423,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551437,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":418006,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3873,"Index":867,"Attempt":0,"Launch Time":1427397551439,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3867,"Index":861,"Attempt":0,"Launch Time":1427397551421,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551439,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":366325,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3874,"Index":868,"Attempt":0,"Launch Time":1427397551440,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3871,"Index":865,"Attempt":0,"Launch Time":1427397551425,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551440,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":14,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":349582,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3875,"Index":869,"Attempt":0,"Launch Time":1427397551443,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3870,"Index":864,"Attempt":0,"Launch Time":1427397551424,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551443,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":6,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":355688,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3876,"Index":870,"Attempt":0,"Launch Time":1427397551445,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3872,"Index":866,"Attempt":0,"Launch Time":1427397551437,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551445,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":345099,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3877,"Index":871,"Attempt":0,"Launch Time":1427397551446,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3873,"Index":867,"Attempt":0,"Launch Time":1427397551439,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551447,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":346653,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3878,"Index":872,"Attempt":0,"Launch Time":1427397551447,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3874,"Index":868,"Attempt":0,"Launch Time":1427397551440,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551447,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":310782,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3879,"Index":873,"Attempt":0,"Launch Time":1427397551450,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3875,"Index":869,"Attempt":0,"Launch Time":1427397551443,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551450,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":327866,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3880,"Index":874,"Attempt":0,"Launch Time":1427397551452,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3876,"Index":870,"Attempt":0,"Launch Time":1427397551445,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551453,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":326380,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3881,"Index":875,"Attempt":0,"Launch Time":1427397551454,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3877,"Index":871,"Attempt":0,"Launch Time":1427397551446,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551455,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":311018,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3882,"Index":876,"Attempt":0,"Launch Time":1427397551456,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3878,"Index":872,"Attempt":0,"Launch Time":1427397551447,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551456,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":339783,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3883,"Index":877,"Attempt":0,"Launch Time":1427397551460,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3880,"Index":874,"Attempt":0,"Launch Time":1427397551452,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551460,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":343241,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3884,"Index":878,"Attempt":0,"Launch Time":1427397551463,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3881,"Index":875,"Attempt":0,"Launch Time":1427397551454,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551463,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":330728,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3885,"Index":879,"Attempt":0,"Launch Time":1427397551464,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3882,"Index":876,"Attempt":0,"Launch Time":1427397551456,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551464,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":336269,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3886,"Index":880,"Attempt":0,"Launch Time":1427397551469,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3879,"Index":873,"Attempt":0,"Launch Time":1427397551450,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551469,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1914941,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3887,"Index":881,"Attempt":0,"Launch Time":1427397551469,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3883,"Index":877,"Attempt":0,"Launch Time":1427397551460,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551470,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":342630,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3888,"Index":882,"Attempt":0,"Launch Time":1427397551471,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3884,"Index":878,"Attempt":0,"Launch Time":1427397551463,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551471,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":319636,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3889,"Index":883,"Attempt":0,"Launch Time":1427397551472,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3885,"Index":879,"Attempt":0,"Launch Time":1427397551464,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551472,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":336287,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3890,"Index":884,"Attempt":0,"Launch Time":1427397551479,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3891,"Index":885,"Attempt":0,"Launch Time":1427397551480,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3887,"Index":881,"Attempt":0,"Launch Time":1427397551469,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551480,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":395064,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3869,"Index":863,"Attempt":0,"Launch Time":1427397551423,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551480,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":51,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":381395,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3892,"Index":886,"Attempt":0,"Launch Time":1427397551480,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3888,"Index":882,"Attempt":0,"Launch Time":1427397551471,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551480,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":421383,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3893,"Index":887,"Attempt":0,"Launch Time":1427397551486,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3830,"Index":824,"Attempt":0,"Launch Time":1427397551337,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551486,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":148,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":450141,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3894,"Index":888,"Attempt":0,"Launch Time":1427397551487,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3839,"Index":833,"Attempt":0,"Launch Time":1427397551359,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551487,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":9,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":306500,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3895,"Index":889,"Attempt":0,"Launch Time":1427397551492,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3893,"Index":887,"Attempt":0,"Launch Time":1427397551486,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551492,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":284206,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3896,"Index":890,"Attempt":0,"Launch Time":1427397551494,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3894,"Index":888,"Attempt":0,"Launch Time":1427397551487,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551494,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":315872,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3897,"Index":891,"Attempt":0,"Launch Time":1427397551497,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3892,"Index":886,"Attempt":0,"Launch Time":1427397551480,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551498,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":5,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":313129,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3898,"Index":892,"Attempt":0,"Launch Time":1427397551499,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3895,"Index":889,"Attempt":0,"Launch Time":1427397551492,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551499,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":293580,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3899,"Index":893,"Attempt":0,"Launch Time":1427397551502,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3896,"Index":890,"Attempt":0,"Launch Time":1427397551494,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551502,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":355629,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3900,"Index":894,"Attempt":0,"Launch Time":1427397551506,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3898,"Index":892,"Attempt":0,"Launch Time":1427397551499,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551506,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":379362,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3901,"Index":895,"Attempt":0,"Launch Time":1427397551510,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3899,"Index":893,"Attempt":0,"Launch Time":1427397551502,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551510,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":285910,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3902,"Index":896,"Attempt":0,"Launch Time":1427397551513,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3897,"Index":891,"Attempt":0,"Launch Time":1427397551497,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551513,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":334387,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3903,"Index":897,"Attempt":0,"Launch Time":1427397551514,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3900,"Index":894,"Attempt":0,"Launch Time":1427397551506,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551514,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":337590,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3904,"Index":898,"Attempt":0,"Launch Time":1427397551518,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3840,"Index":834,"Attempt":0,"Launch Time":1427397551360,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551518,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":41,"Executor Run Time":25,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":388979,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3905,"Index":899,"Attempt":0,"Launch Time":1427397551518,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3901,"Index":895,"Attempt":0,"Launch Time":1427397551510,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551519,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":390905,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3906,"Index":900,"Attempt":0,"Launch Time":1427397551521,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3903,"Index":897,"Attempt":0,"Launch Time":1427397551514,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551521,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":341634,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3907,"Index":901,"Attempt":0,"Launch Time":1427397551522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3902,"Index":896,"Attempt":0,"Launch Time":1427397551513,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551522,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":325476,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3908,"Index":902,"Attempt":0,"Launch Time":1427397551526,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3904,"Index":898,"Attempt":0,"Launch Time":1427397551518,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551526,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":335685,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3909,"Index":903,"Attempt":0,"Launch Time":1427397551526,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3905,"Index":899,"Attempt":0,"Launch Time":1427397551518,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551526,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":348629,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3910,"Index":904,"Attempt":0,"Launch Time":1427397551529,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3906,"Index":900,"Attempt":0,"Launch Time":1427397551521,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551529,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":411220,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3911,"Index":905,"Attempt":0,"Launch Time":1427397551534,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3908,"Index":902,"Attempt":0,"Launch Time":1427397551526,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551534,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":366297,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3912,"Index":906,"Attempt":0,"Launch Time":1427397551534,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3907,"Index":901,"Attempt":0,"Launch Time":1427397551522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551534,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":2521253,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3913,"Index":907,"Attempt":0,"Launch Time":1427397551535,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3909,"Index":903,"Attempt":0,"Launch Time":1427397551526,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551535,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9608,"Shuffle Write Time":459258,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3914,"Index":908,"Attempt":0,"Launch Time":1427397551537,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3886,"Index":880,"Attempt":0,"Launch Time":1427397551469,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551537,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":68,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":4180501,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3910,"Index":904,"Attempt":0,"Launch Time":1427397551529,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551537,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":355708,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3915,"Index":909,"Attempt":0,"Launch Time":1427397551537,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3916,"Index":910,"Attempt":0,"Launch Time":1427397551547,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3889,"Index":883,"Attempt":0,"Launch Time":1427397551472,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551547,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":75,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":533167,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3917,"Index":911,"Attempt":0,"Launch Time":1427397551549,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3912,"Index":906,"Attempt":0,"Launch Time":1427397551534,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551549,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":14,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":337488,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3918,"Index":912,"Attempt":0,"Launch Time":1427397551550,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3913,"Index":907,"Attempt":0,"Launch Time":1427397551535,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551550,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":350715,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3919,"Index":913,"Attempt":0,"Launch Time":1427397551551,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3911,"Index":905,"Attempt":0,"Launch Time":1427397551534,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551551,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":336093,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3920,"Index":914,"Attempt":0,"Launch Time":1427397551557,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3917,"Index":911,"Attempt":0,"Launch Time":1427397551549,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551557,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":340754,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3921,"Index":915,"Attempt":0,"Launch Time":1427397551557,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3918,"Index":912,"Attempt":0,"Launch Time":1427397551550,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551557,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":357416,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3922,"Index":916,"Attempt":0,"Launch Time":1427397551558,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3919,"Index":913,"Attempt":0,"Launch Time":1427397551551,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551558,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":343356,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3923,"Index":917,"Attempt":0,"Launch Time":1427397551564,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3920,"Index":914,"Attempt":0,"Launch Time":1427397551557,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551565,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":355622,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3924,"Index":918,"Attempt":0,"Launch Time":1427397551565,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3921,"Index":915,"Attempt":0,"Launch Time":1427397551557,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551565,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":346929,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3925,"Index":919,"Attempt":0,"Launch Time":1427397551565,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3922,"Index":916,"Attempt":0,"Launch Time":1427397551558,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551565,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":343398,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3926,"Index":920,"Attempt":0,"Launch Time":1427397551568,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3891,"Index":885,"Attempt":0,"Launch Time":1427397551480,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551568,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":29,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":369699,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3927,"Index":921,"Attempt":0,"Launch Time":1427397551572,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3924,"Index":918,"Attempt":0,"Launch Time":1427397551565,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551572,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":333963,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3928,"Index":922,"Attempt":0,"Launch Time":1427397551574,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3923,"Index":917,"Attempt":0,"Launch Time":1427397551564,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551574,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":670768,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3929,"Index":923,"Attempt":0,"Launch Time":1427397551574,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3925,"Index":919,"Attempt":0,"Launch Time":1427397551565,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551574,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":351842,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3930,"Index":924,"Attempt":0,"Launch Time":1427397551575,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3916,"Index":910,"Attempt":0,"Launch Time":1427397551547,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551575,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":325226,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3931,"Index":925,"Attempt":0,"Launch Time":1427397551579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3915,"Index":909,"Attempt":0,"Launch Time":1427397551537,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551580,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":334823,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3932,"Index":926,"Attempt":0,"Launch Time":1427397551580,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3927,"Index":921,"Attempt":0,"Launch Time":1427397551572,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551580,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":314322,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3933,"Index":927,"Attempt":0,"Launch Time":1427397551581,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3928,"Index":922,"Attempt":0,"Launch Time":1427397551574,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551581,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":327491,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3934,"Index":928,"Attempt":0,"Launch Time":1427397551584,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3929,"Index":923,"Attempt":0,"Launch Time":1427397551574,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551584,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":334077,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3935,"Index":929,"Attempt":0,"Launch Time":1427397551587,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3932,"Index":926,"Attempt":0,"Launch Time":1427397551580,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551587,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":302674,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3936,"Index":930,"Attempt":0,"Launch Time":1427397551588,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3933,"Index":927,"Attempt":0,"Launch Time":1427397551581,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551588,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":345525,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3937,"Index":931,"Attempt":0,"Launch Time":1427397551591,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3934,"Index":928,"Attempt":0,"Launch Time":1427397551584,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551591,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":287689,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3938,"Index":932,"Attempt":0,"Launch Time":1427397551593,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3914,"Index":908,"Attempt":0,"Launch Time":1427397551537,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551593,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":56,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":328123,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3939,"Index":933,"Attempt":0,"Launch Time":1427397551601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3935,"Index":929,"Attempt":0,"Launch Time":1427397551587,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551607,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":318964,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3940,"Index":934,"Attempt":0,"Launch Time":1427397551610,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3938,"Index":932,"Attempt":0,"Launch Time":1427397551593,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551610,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":356053,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3941,"Index":935,"Attempt":0,"Launch Time":1427397551611,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3942,"Index":936,"Attempt":0,"Launch Time":1427397551611,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3943,"Index":937,"Attempt":0,"Launch Time":1427397551611,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3937,"Index":931,"Attempt":0,"Launch Time":1427397551591,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551611,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":327305,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3944,"Index":938,"Attempt":0,"Launch Time":1427397551611,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3926,"Index":920,"Attempt":0,"Launch Time":1427397551568,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551611,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":22,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":362545,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3890,"Index":884,"Attempt":0,"Launch Time":1427397551479,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551612,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":65,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1488130,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3936,"Index":930,"Attempt":0,"Launch Time":1427397551588,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551612,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":316481,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3945,"Index":939,"Attempt":0,"Launch Time":1427397551620,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3931,"Index":925,"Attempt":0,"Launch Time":1427397551579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551620,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":13,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":353637,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3941,"Index":935,"Attempt":0,"Launch Time":1427397551611,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551623,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":320568,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3946,"Index":940,"Attempt":0,"Launch Time":1427397551623,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3947,"Index":941,"Attempt":0,"Launch Time":1427397551626,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3945,"Index":939,"Attempt":0,"Launch Time":1427397551620,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551627,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":270821,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3948,"Index":942,"Attempt":0,"Launch Time":1427397551627,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3944,"Index":938,"Attempt":0,"Launch Time":1427397551611,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551627,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":6,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":312902,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3949,"Index":943,"Attempt":0,"Launch Time":1427397551628,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3940,"Index":934,"Attempt":0,"Launch Time":1427397551610,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551628,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":298923,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3950,"Index":944,"Attempt":0,"Launch Time":1427397551629,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3930,"Index":924,"Attempt":0,"Launch Time":1427397551575,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551629,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":14,"Executor Run Time":20,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":313015,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3951,"Index":945,"Attempt":0,"Launch Time":1427397551634,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3947,"Index":941,"Attempt":0,"Launch Time":1427397551626,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551634,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":300067,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3952,"Index":946,"Attempt":0,"Launch Time":1427397551636,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3946,"Index":940,"Attempt":0,"Launch Time":1427397551623,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551636,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":326017,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3953,"Index":947,"Attempt":0,"Launch Time":1427397551638,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3949,"Index":943,"Attempt":0,"Launch Time":1427397551628,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551638,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":309651,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3954,"Index":948,"Attempt":0,"Launch Time":1427397551642,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3951,"Index":945,"Attempt":0,"Launch Time":1427397551634,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551642,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":339198,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3955,"Index":949,"Attempt":0,"Launch Time":1427397551645,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3950,"Index":944,"Attempt":0,"Launch Time":1427397551629,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551645,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":343365,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3956,"Index":950,"Attempt":0,"Launch Time":1427397551649,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3952,"Index":946,"Attempt":0,"Launch Time":1427397551636,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551649,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":364780,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3957,"Index":951,"Attempt":0,"Launch Time":1427397551649,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3954,"Index":948,"Attempt":0,"Launch Time":1427397551642,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551649,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":353037,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3958,"Index":952,"Attempt":0,"Launch Time":1427397551650,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3953,"Index":947,"Attempt":0,"Launch Time":1427397551638,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551650,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":2462476,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3959,"Index":953,"Attempt":0,"Launch Time":1427397551653,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3948,"Index":942,"Attempt":0,"Launch Time":1427397551627,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551653,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":24,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1047171,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3960,"Index":954,"Attempt":0,"Launch Time":1427397551662,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3958,"Index":952,"Attempt":0,"Launch Time":1427397551650,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551662,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":359646,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3961,"Index":955,"Attempt":0,"Launch Time":1427397551663,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3957,"Index":951,"Attempt":0,"Launch Time":1427397551649,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551663,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":385485,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3962,"Index":956,"Attempt":0,"Launch Time":1427397551665,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3963,"Index":957,"Attempt":0,"Launch Time":1427397551665,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3955,"Index":949,"Attempt":0,"Launch Time":1427397551645,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551665,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":370681,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3956,"Index":950,"Attempt":0,"Launch Time":1427397551649,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551665,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":319858,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3964,"Index":958,"Attempt":0,"Launch Time":1427397551674,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3939,"Index":933,"Attempt":0,"Launch Time":1427397551601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551674,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":53,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":504765,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3965,"Index":959,"Attempt":0,"Launch Time":1427397551680,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3961,"Index":955,"Attempt":0,"Launch Time":1427397551663,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551680,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":16,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":330182,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3966,"Index":960,"Attempt":0,"Launch Time":1427397551682,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3959,"Index":953,"Attempt":0,"Launch Time":1427397551653,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551682,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":9,"Executor Run Time":18,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":348993,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3967,"Index":961,"Attempt":0,"Launch Time":1427397551689,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3942,"Index":936,"Attempt":0,"Launch Time":1427397551611,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551689,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":9,"Executor Run Time":23,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1502164,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3968,"Index":962,"Attempt":0,"Launch Time":1427397551691,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3966,"Index":960,"Attempt":0,"Launch Time":1427397551682,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551691,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":348233,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3969,"Index":963,"Attempt":0,"Launch Time":1427397551696,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3967,"Index":961,"Attempt":0,"Launch Time":1427397551689,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551696,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":352255,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3970,"Index":964,"Attempt":0,"Launch Time":1427397551697,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3968,"Index":962,"Attempt":0,"Launch Time":1427397551691,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551697,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":326615,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3971,"Index":965,"Attempt":0,"Launch Time":1427397551704,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3972,"Index":966,"Attempt":0,"Launch Time":1427397551704,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3960,"Index":954,"Attempt":0,"Launch Time":1427397551662,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551704,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":31,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":535917,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3962,"Index":956,"Attempt":0,"Launch Time":1427397551665,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551704,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":9,"Executor Run Time":30,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":361531,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3973,"Index":967,"Attempt":0,"Launch Time":1427397551705,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3970,"Index":964,"Attempt":0,"Launch Time":1427397551697,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551705,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":380742,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3974,"Index":968,"Attempt":0,"Launch Time":1427397551705,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3963,"Index":957,"Attempt":0,"Launch Time":1427397551665,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551705,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":30,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":400756,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3975,"Index":969,"Attempt":0,"Launch Time":1427397551705,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3969,"Index":963,"Attempt":0,"Launch Time":1427397551696,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551705,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":418545,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3976,"Index":970,"Attempt":0,"Launch Time":1427397551709,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3964,"Index":958,"Attempt":0,"Launch Time":1427397551674,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551709,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":27,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":830920,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3977,"Index":971,"Attempt":0,"Launch Time":1427397551709,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3943,"Index":937,"Attempt":0,"Launch Time":1427397551611,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551710,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":17,"Executor Run Time":30,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":349864,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3978,"Index":972,"Attempt":0,"Launch Time":1427397551713,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3973,"Index":967,"Attempt":0,"Launch Time":1427397551705,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551713,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":378855,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3979,"Index":973,"Attempt":0,"Launch Time":1427397551714,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3972,"Index":966,"Attempt":0,"Launch Time":1427397551704,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551714,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":373817,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3980,"Index":974,"Attempt":0,"Launch Time":1427397551714,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3971,"Index":965,"Attempt":0,"Launch Time":1427397551704,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551714,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":445079,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3981,"Index":975,"Attempt":0,"Launch Time":1427397551715,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3965,"Index":959,"Attempt":0,"Launch Time":1427397551680,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551715,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":22,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":348628,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3982,"Index":976,"Attempt":0,"Launch Time":1427397551717,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3974,"Index":968,"Attempt":0,"Launch Time":1427397551705,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551717,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":541572,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3983,"Index":977,"Attempt":0,"Launch Time":1427397551722,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3978,"Index":972,"Attempt":0,"Launch Time":1427397551713,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551722,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":323758,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3984,"Index":978,"Attempt":0,"Launch Time":1427397551723,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3980,"Index":974,"Attempt":0,"Launch Time":1427397551714,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551723,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":322703,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3985,"Index":979,"Attempt":0,"Launch Time":1427397551723,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3986,"Index":980,"Attempt":0,"Launch Time":1427397551723,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3981,"Index":975,"Attempt":0,"Launch Time":1427397551715,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551723,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":338354,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3979,"Index":973,"Attempt":0,"Launch Time":1427397551714,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551723,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":378392,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3987,"Index":981,"Attempt":0,"Launch Time":1427397551726,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3976,"Index":970,"Attempt":0,"Launch Time":1427397551709,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551726,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":3346030,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3988,"Index":982,"Attempt":0,"Launch Time":1427397551731,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3989,"Index":983,"Attempt":0,"Launch Time":1427397551731,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3986,"Index":980,"Attempt":0,"Launch Time":1427397551723,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551731,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":343820,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3985,"Index":979,"Attempt":0,"Launch Time":1427397551723,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551731,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":553017,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3990,"Index":984,"Attempt":0,"Launch Time":1427397551736,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3987,"Index":981,"Attempt":0,"Launch Time":1427397551726,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551736,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":297158,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3991,"Index":985,"Attempt":0,"Launch Time":1427397551738,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3988,"Index":982,"Attempt":0,"Launch Time":1427397551731,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551738,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":302128,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3992,"Index":986,"Attempt":0,"Launch Time":1427397551738,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3989,"Index":983,"Attempt":0,"Launch Time":1427397551731,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551739,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":351861,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3993,"Index":987,"Attempt":0,"Launch Time":1427397551744,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3990,"Index":984,"Attempt":0,"Launch Time":1427397551736,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551744,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":290684,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3994,"Index":988,"Attempt":0,"Launch Time":1427397551744,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3975,"Index":969,"Attempt":0,"Launch Time":1427397551705,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551744,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":38,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":316807,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3995,"Index":989,"Attempt":0,"Launch Time":1427397551750,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3984,"Index":978,"Attempt":0,"Launch Time":1427397551723,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551750,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":26,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":380037,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3996,"Index":990,"Attempt":0,"Launch Time":1427397551750,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3977,"Index":971,"Attempt":0,"Launch Time":1427397551709,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551750,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":25,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":359340,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3997,"Index":991,"Attempt":0,"Launch Time":1427397551751,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3983,"Index":977,"Attempt":0,"Launch Time":1427397551722,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551751,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":27,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":362230,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3998,"Index":992,"Attempt":0,"Launch Time":1427397551751,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3993,"Index":987,"Attempt":0,"Launch Time":1427397551744,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551752,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":341859,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3994,"Index":988,"Attempt":0,"Launch Time":1427397551744,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551752,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":346697,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3999,"Index":993,"Attempt":0,"Launch Time":1427397551752,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":4000,"Index":994,"Attempt":0,"Launch Time":1427397551757,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3995,"Index":989,"Attempt":0,"Launch Time":1427397551750,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551758,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":360407,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":4001,"Index":995,"Attempt":0,"Launch Time":1427397551759,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3998,"Index":992,"Attempt":0,"Launch Time":1427397551751,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551759,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":349489,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":4002,"Index":996,"Attempt":0,"Launch Time":1427397551759,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3999,"Index":993,"Attempt":0,"Launch Time":1427397551752,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551759,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":375287,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":4003,"Index":997,"Attempt":0,"Launch Time":1427397551760,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3996,"Index":990,"Attempt":0,"Launch Time":1427397551750,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551760,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":359503,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":4004,"Index":998,"Attempt":0,"Launch Time":1427397551766,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4000,"Index":994,"Attempt":0,"Launch Time":1427397551757,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551766,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":318496,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":4005,"Index":999,"Attempt":0,"Launch Time":1427397551768,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4002,"Index":996,"Attempt":0,"Launch Time":1427397551759,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551768,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":381403,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4001,"Index":995,"Attempt":0,"Launch Time":1427397551759,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551769,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":397252,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4003,"Index":997,"Attempt":0,"Launch Time":1427397551760,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551769,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":375553,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3997,"Index":991,"Attempt":0,"Launch Time":1427397551751,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551778,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":415810,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3991,"Index":985,"Attempt":0,"Launch Time":1427397551738,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551779,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":40,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":373550,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3982,"Index":976,"Attempt":0,"Launch Time":1427397551717,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551782,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":63,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":330556,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4005,"Index":999,"Attempt":0,"Launch Time":1427397551768,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551782,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":291204,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3992,"Index":986,"Attempt":0,"Launch Time":1427397551738,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551783,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":43,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":426007,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4004,"Index":998,"Attempt":0,"Launch Time":1427397551766,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551784,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":331454,"Shuffle Records Written":100}}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":12,"Stage Attempt ID":0,"Stage Name":"map at :21","Number of Tasks":1000,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:287)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:21)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:28)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:30)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.(:32)\n$line15.$read$$iwC$$iwC$$iwC.(:34)\n$line15.$read$$iwC$$iwC.(:36)\n$line15.$read$$iwC.(:38)\n$line15.$read.(:40)\n$line15.$read$.(:44)\n$line15.$read$.()\n$line15.$eval$.(:7)\n$line15.$eval$.()\n$line15.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Submission Time":1427397549495,"Completion Time":1427397551784,"Accumulables":[]}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":13,"Stage Attempt ID":0,"Stage Name":"next at :26","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":50,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"scala.collection.Iterator$$anon$13.next(Iterator.scala:372)\n$line32.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line32.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:31)\n$line32.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:33)\n$line32.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:35)\n$line32.$read$$iwC$$iwC$$iwC$$iwC.(:37)\n$line32.$read$$iwC$$iwC$$iwC.(:39)\n$line32.$read$$iwC$$iwC.(:41)\n$line32.$read$$iwC.(:43)\n$line32.$read.(:45)\n$line32.$read$.(:49)\n$line32.$read$.()\n$line32.$eval$.(:7)\n$line32.$eval$.()\n$line32.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":13,"Stage Attempt ID":0,"Task Info":{"Task ID":4006,"Index":0,"Attempt":0,"Launch Time":1427397551788,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":13,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4006,"Index":0,"Attempt":0,"Launch Time":1427397551788,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551931,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":142,"Result Size":1060,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":1000,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":192090,"Total Records Read":2000}}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":13,"Stage Attempt ID":0,"Stage Name":"next at :26","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":50,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"scala.collection.Iterator$$anon$13.next(Iterator.scala:372)\n$line32.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line32.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:31)\n$line32.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:33)\n$line32.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:35)\n$line32.$read$$iwC$$iwC$$iwC$$iwC.(:37)\n$line32.$read$$iwC$$iwC$$iwC.(:39)\n$line32.$read$$iwC$$iwC.(:41)\n$line32.$read$$iwC.(:43)\n$line32.$read.(:45)\n$line32.$read$.(:49)\n$line32.$read$.()\n$line32.$eval$.(:7)\n$line32.$eval$.()\n$line32.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Submission Time":1427397551788,"Completion Time":1427397551931,"Accumulables":[]}} -{"Event":"SparkListenerJobEnd","Job ID":6,"Completion Time":1427397551931,"Job Result":{"Result":"JobSucceeded"}} -{"Event":"SparkListenerJobStart","Job ID":7,"Submission Time":1427397553570,"Stage Infos":[{"Stage ID":15,"Stage Attempt ID":0,"Stage Name":"next at :26","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":50,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"scala.collection.Iterator$$anon$13.next(Iterator.scala:372)\n$line34.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line34.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:31)\n$line34.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:33)\n$line34.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:35)\n$line34.$read$$iwC$$iwC$$iwC$$iwC.(:37)\n$line34.$read$$iwC$$iwC$$iwC.(:39)\n$line34.$read$$iwC$$iwC.(:41)\n$line34.$read$$iwC.(:43)\n$line34.$read.(:45)\n$line34.$read$.(:49)\n$line34.$read$.()\n$line34.$eval$.(:7)\n$line34.$eval$.()\n$line34.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]},{"Stage ID":14,"Stage Attempt ID":0,"Stage Name":"map at :21","Number of Tasks":1000,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:287)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:21)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:28)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:30)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.(:32)\n$line15.$read$$iwC$$iwC$$iwC.(:34)\n$line15.$read$$iwC$$iwC.(:36)\n$line15.$read$$iwC.(:38)\n$line15.$read.(:40)\n$line15.$read$.(:44)\n$line15.$read$.()\n$line15.$eval$.(:7)\n$line15.$eval$.()\n$line15.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]}],"Stage IDs":[15,14]} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":15,"Stage Attempt ID":0,"Stage Name":"next at :26","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":50,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"scala.collection.Iterator$$anon$13.next(Iterator.scala:372)\n$line34.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line34.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:31)\n$line34.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:33)\n$line34.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:35)\n$line34.$read$$iwC$$iwC$$iwC$$iwC.(:37)\n$line34.$read$$iwC$$iwC$$iwC.(:39)\n$line34.$read$$iwC$$iwC.(:41)\n$line34.$read$$iwC.(:43)\n$line34.$read.(:45)\n$line34.$read$.(:49)\n$line34.$read$.()\n$line34.$eval$.(:7)\n$line34.$eval$.()\n$line34.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":15,"Stage Attempt ID":0,"Task Info":{"Task ID":4007,"Index":0,"Attempt":0,"Launch Time":1427397553573,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":15,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4007,"Index":0,"Attempt":0,"Launch Time":1427397553573,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397553687,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":112,"Result Size":1060,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":1000,"Fetch Wait Time":1,"Remote Bytes Read":0,"Local Bytes Read":192017,"Total Records Read":2000}}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":15,"Stage Attempt ID":0,"Stage Name":"next at :26","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":50,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"scala.collection.Iterator$$anon$13.next(Iterator.scala:372)\n$line34.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line34.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:31)\n$line34.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:33)\n$line34.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:35)\n$line34.$read$$iwC$$iwC$$iwC$$iwC.(:37)\n$line34.$read$$iwC$$iwC$$iwC.(:39)\n$line34.$read$$iwC$$iwC.(:41)\n$line34.$read$$iwC.(:43)\n$line34.$read.(:45)\n$line34.$read$.(:49)\n$line34.$read$.()\n$line34.$eval$.(:7)\n$line34.$eval$.()\n$line34.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Submission Time":1427397553573,"Completion Time":1427397553687,"Accumulables":[]}} -{"Event":"SparkListenerJobEnd","Job ID":7,"Completion Time":1427397553687,"Job Result":{"Result":"JobSucceeded"}} -{"Event":"SparkListenerJobStart","Job ID":8,"Submission Time":1427397593688,"Stage Infos":[{"Stage ID":16,"Stage Attempt ID":0,"Stage Name":"map at :21","Number of Tasks":1000,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:287)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:21)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:28)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:30)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.(:32)\n$line15.$read$$iwC$$iwC$$iwC.(:34)\n$line15.$read$$iwC$$iwC.(:36)\n$line15.$read$$iwC.(:38)\n$line15.$read.(:40)\n$line15.$read$.(:44)\n$line15.$read$.()\n$line15.$eval$.(:7)\n$line15.$eval$.()\n$line15.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]},{"Stage ID":17,"Stage Attempt ID":0,"Stage Name":"next at :26","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":50,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"scala.collection.Iterator$$anon$13.next(Iterator.scala:372)\n$line36.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line36.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:31)\n$line36.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:33)\n$line36.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:35)\n$line36.$read$$iwC$$iwC$$iwC$$iwC.(:37)\n$line36.$read$$iwC$$iwC$$iwC.(:39)\n$line36.$read$$iwC$$iwC.(:41)\n$line36.$read$$iwC.(:43)\n$line36.$read.(:45)\n$line36.$read$.(:49)\n$line36.$read$.()\n$line36.$eval$.(:7)\n$line36.$eval$.()\n$line36.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]}],"Stage IDs":[16,17]} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":16,"Stage Attempt ID":0,"Stage Name":"map at :21","Number of Tasks":1000,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:287)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:21)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:28)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:30)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.(:32)\n$line15.$read$$iwC$$iwC$$iwC.(:34)\n$line15.$read$$iwC$$iwC.(:36)\n$line15.$read$$iwC.(:38)\n$line15.$read.(:40)\n$line15.$read$.(:44)\n$line15.$read$.()\n$line15.$eval$.(:7)\n$line15.$eval$.()\n$line15.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4008,"Index":0,"Attempt":0,"Launch Time":1427397593695,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4009,"Index":1,"Attempt":0,"Launch Time":1427397593695,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4010,"Index":2,"Attempt":0,"Launch Time":1427397593695,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4011,"Index":3,"Attempt":0,"Launch Time":1427397593695,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4012,"Index":4,"Attempt":0,"Launch Time":1427397593695,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4013,"Index":5,"Attempt":0,"Launch Time":1427397593695,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4014,"Index":6,"Attempt":0,"Launch Time":1427397593695,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4015,"Index":7,"Attempt":0,"Launch Time":1427397593695,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4016,"Index":8,"Attempt":0,"Launch Time":1427397593702,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4017,"Index":9,"Attempt":0,"Launch Time":1427397593702,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4012,"Index":4,"Attempt":0,"Launch Time":1427397593695,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593702,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":287332,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4018,"Index":10,"Attempt":0,"Launch Time":1427397593704,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4019,"Index":11,"Attempt":0,"Launch Time":1427397593705,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4020,"Index":12,"Attempt":0,"Launch Time":1427397593705,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4008,"Index":0,"Attempt":0,"Launch Time":1427397593695,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593706,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":239517,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4021,"Index":13,"Attempt":0,"Launch Time":1427397593709,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4009,"Index":1,"Attempt":0,"Launch Time":1427397593695,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593709,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":269976,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4010,"Index":2,"Attempt":0,"Launch Time":1427397593695,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593709,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":569524,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4014,"Index":6,"Attempt":0,"Launch Time":1427397593695,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593709,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":260818,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4013,"Index":5,"Attempt":0,"Launch Time":1427397593695,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593709,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":281354,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4022,"Index":14,"Attempt":0,"Launch Time":1427397593710,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4023,"Index":15,"Attempt":0,"Launch Time":1427397593710,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4024,"Index":16,"Attempt":0,"Launch Time":1427397593710,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4011,"Index":3,"Attempt":0,"Launch Time":1427397593695,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593710,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":268311,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4015,"Index":7,"Attempt":0,"Launch Time":1427397593695,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593710,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":267486,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4016,"Index":8,"Attempt":0,"Launch Time":1427397593702,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593710,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":261740,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4025,"Index":17,"Attempt":0,"Launch Time":1427397593712,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4017,"Index":9,"Attempt":0,"Launch Time":1427397593702,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593712,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":285963,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4026,"Index":18,"Attempt":0,"Launch Time":1427397593713,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4020,"Index":12,"Attempt":0,"Launch Time":1427397593705,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593714,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":270373,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4027,"Index":19,"Attempt":0,"Launch Time":1427397593714,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4018,"Index":10,"Attempt":0,"Launch Time":1427397593704,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593715,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":281378,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4028,"Index":20,"Attempt":0,"Launch Time":1427397593717,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4019,"Index":11,"Attempt":0,"Launch Time":1427397593705,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593717,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":896763,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4029,"Index":21,"Attempt":0,"Launch Time":1427397593718,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4030,"Index":22,"Attempt":0,"Launch Time":1427397593718,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4024,"Index":16,"Attempt":0,"Launch Time":1427397593710,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593718,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":259127,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4021,"Index":13,"Attempt":0,"Launch Time":1427397593709,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593719,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":528453,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4031,"Index":23,"Attempt":0,"Launch Time":1427397593719,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4032,"Index":24,"Attempt":0,"Launch Time":1427397593719,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4033,"Index":25,"Attempt":0,"Launch Time":1427397593719,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4022,"Index":14,"Attempt":0,"Launch Time":1427397593710,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593720,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":491624,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4023,"Index":15,"Attempt":0,"Launch Time":1427397593710,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593722,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":587498,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4034,"Index":26,"Attempt":0,"Launch Time":1427397593722,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4035,"Index":27,"Attempt":0,"Launch Time":1427397593722,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4025,"Index":17,"Attempt":0,"Launch Time":1427397593712,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593723,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":491089,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4027,"Index":19,"Attempt":0,"Launch Time":1427397593714,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593737,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":301015,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4026,"Index":18,"Attempt":0,"Launch Time":1427397593713,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593738,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":310954,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4036,"Index":28,"Attempt":0,"Launch Time":1427397593740,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4028,"Index":20,"Attempt":0,"Launch Time":1427397593717,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593740,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":20,"Result Size":930,"JVM GC Time":13,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":299714,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4037,"Index":29,"Attempt":0,"Launch Time":1427397593740,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4038,"Index":30,"Attempt":0,"Launch Time":1427397593740,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4030,"Index":22,"Attempt":0,"Launch Time":1427397593718,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593741,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":19,"Result Size":930,"JVM GC Time":13,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":292926,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4029,"Index":21,"Attempt":0,"Launch Time":1427397593718,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593741,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":13,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":264415,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4039,"Index":31,"Attempt":0,"Launch Time":1427397593747,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4040,"Index":32,"Attempt":0,"Launch Time":1427397593747,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4041,"Index":33,"Attempt":0,"Launch Time":1427397593747,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4036,"Index":28,"Attempt":0,"Launch Time":1427397593740,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593747,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":268786,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4037,"Index":29,"Attempt":0,"Launch Time":1427397593740,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593747,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":262819,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4042,"Index":34,"Attempt":0,"Launch Time":1427397593747,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4043,"Index":35,"Attempt":0,"Launch Time":1427397593747,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4038,"Index":30,"Attempt":0,"Launch Time":1427397593740,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593748,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":240790,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4035,"Index":27,"Attempt":0,"Launch Time":1427397593722,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593748,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":17,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":290601,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4034,"Index":26,"Attempt":0,"Launch Time":1427397593722,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593748,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":24,"Result Size":930,"JVM GC Time":13,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":296886,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4044,"Index":36,"Attempt":0,"Launch Time":1427397593754,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4045,"Index":37,"Attempt":0,"Launch Time":1427397593754,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4046,"Index":38,"Attempt":0,"Launch Time":1427397593754,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4039,"Index":31,"Attempt":0,"Launch Time":1427397593747,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593754,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":359948,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4040,"Index":32,"Attempt":0,"Launch Time":1427397593747,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593755,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":271270,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4047,"Index":39,"Attempt":0,"Launch Time":1427397593755,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4048,"Index":40,"Attempt":0,"Launch Time":1427397593755,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4042,"Index":34,"Attempt":0,"Launch Time":1427397593747,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593756,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":571799,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4043,"Index":35,"Attempt":0,"Launch Time":1427397593747,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593756,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":264174,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4041,"Index":33,"Attempt":0,"Launch Time":1427397593747,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593756,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":356384,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4049,"Index":41,"Attempt":0,"Launch Time":1427397593760,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4044,"Index":36,"Attempt":0,"Launch Time":1427397593754,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593761,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":283685,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4050,"Index":42,"Attempt":0,"Launch Time":1427397593761,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4051,"Index":43,"Attempt":0,"Launch Time":1427397593761,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4046,"Index":38,"Attempt":0,"Launch Time":1427397593754,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593761,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":264052,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4045,"Index":37,"Attempt":0,"Launch Time":1427397593754,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593761,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":271026,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4052,"Index":44,"Attempt":0,"Launch Time":1427397593762,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4048,"Index":40,"Attempt":0,"Launch Time":1427397593755,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593762,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":301676,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4053,"Index":45,"Attempt":0,"Launch Time":1427397593763,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4047,"Index":39,"Attempt":0,"Launch Time":1427397593755,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593763,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":286825,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4054,"Index":46,"Attempt":0,"Launch Time":1427397593767,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4049,"Index":41,"Attempt":0,"Launch Time":1427397593760,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593767,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":341656,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4055,"Index":47,"Attempt":0,"Launch Time":1427397593767,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4050,"Index":42,"Attempt":0,"Launch Time":1427397593761,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593767,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":321407,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4056,"Index":48,"Attempt":0,"Launch Time":1427397593768,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4051,"Index":43,"Attempt":0,"Launch Time":1427397593761,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593768,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":307576,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4057,"Index":49,"Attempt":0,"Launch Time":1427397593769,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4053,"Index":45,"Attempt":0,"Launch Time":1427397593763,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593769,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":289841,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4058,"Index":50,"Attempt":0,"Launch Time":1427397593775,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4052,"Index":44,"Attempt":0,"Launch Time":1427397593762,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593775,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":289606,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4059,"Index":51,"Attempt":0,"Launch Time":1427397593776,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4057,"Index":49,"Attempt":0,"Launch Time":1427397593769,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593776,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":253573,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4060,"Index":52,"Attempt":0,"Launch Time":1427397593778,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4056,"Index":48,"Attempt":0,"Launch Time":1427397593768,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593778,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":277426,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4061,"Index":53,"Attempt":0,"Launch Time":1427397593782,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4058,"Index":50,"Attempt":0,"Launch Time":1427397593775,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593782,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":303835,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4062,"Index":54,"Attempt":0,"Launch Time":1427397593782,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4059,"Index":51,"Attempt":0,"Launch Time":1427397593776,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593783,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":278696,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4063,"Index":55,"Attempt":0,"Launch Time":1427397593783,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4054,"Index":46,"Attempt":0,"Launch Time":1427397593767,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593783,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":311122,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4064,"Index":56,"Attempt":0,"Launch Time":1427397593789,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4061,"Index":53,"Attempt":0,"Launch Time":1427397593782,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593789,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":330279,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4065,"Index":57,"Attempt":0,"Launch Time":1427397593791,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4055,"Index":47,"Attempt":0,"Launch Time":1427397593767,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593791,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":19,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":611625,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4066,"Index":58,"Attempt":0,"Launch Time":1427397593792,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4062,"Index":54,"Attempt":0,"Launch Time":1427397593782,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593792,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":641167,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4067,"Index":59,"Attempt":0,"Launch Time":1427397593792,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4063,"Index":55,"Attempt":0,"Launch Time":1427397593783,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593793,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":275692,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4068,"Index":60,"Attempt":0,"Launch Time":1427397593794,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4060,"Index":52,"Attempt":0,"Launch Time":1427397593778,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593794,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1067821,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4069,"Index":61,"Attempt":0,"Launch Time":1427397593798,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4064,"Index":56,"Attempt":0,"Launch Time":1427397593789,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593798,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":373084,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4070,"Index":62,"Attempt":0,"Launch Time":1427397593798,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4065,"Index":57,"Attempt":0,"Launch Time":1427397593791,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593798,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":419259,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4071,"Index":63,"Attempt":0,"Launch Time":1427397593799,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4066,"Index":58,"Attempt":0,"Launch Time":1427397593792,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593799,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":252889,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4072,"Index":64,"Attempt":0,"Launch Time":1427397593800,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4067,"Index":59,"Attempt":0,"Launch Time":1427397593792,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593800,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9610,"Shuffle Write Time":272564,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4073,"Index":65,"Attempt":0,"Launch Time":1427397593804,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4068,"Index":60,"Attempt":0,"Launch Time":1427397593794,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593804,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":495597,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4074,"Index":66,"Attempt":0,"Launch Time":1427397593806,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4071,"Index":63,"Attempt":0,"Launch Time":1427397593799,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593806,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":277942,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4075,"Index":67,"Attempt":0,"Launch Time":1427397593806,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4069,"Index":61,"Attempt":0,"Launch Time":1427397593798,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593806,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":256885,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4076,"Index":68,"Attempt":0,"Launch Time":1427397593806,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4072,"Index":64,"Attempt":0,"Launch Time":1427397593800,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593806,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":272910,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4077,"Index":69,"Attempt":0,"Launch Time":1427397593807,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4031,"Index":23,"Attempt":0,"Launch Time":1427397593719,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593808,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":88,"Result Size":930,"JVM GC Time":13,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":349773,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4078,"Index":70,"Attempt":0,"Launch Time":1427397593808,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4032,"Index":24,"Attempt":0,"Launch Time":1427397593719,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593808,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":88,"Result Size":930,"JVM GC Time":13,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":353467,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4079,"Index":71,"Attempt":0,"Launch Time":1427397593809,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4070,"Index":62,"Attempt":0,"Launch Time":1427397593798,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593809,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":346074,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4080,"Index":72,"Attempt":0,"Launch Time":1427397593809,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4033,"Index":25,"Attempt":0,"Launch Time":1427397593719,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593810,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":89,"Result Size":930,"JVM GC Time":13,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":326875,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4081,"Index":73,"Attempt":0,"Launch Time":1427397593820,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4075,"Index":67,"Attempt":0,"Launch Time":1427397593806,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593820,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9614,"Shuffle Write Time":266058,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4082,"Index":74,"Attempt":0,"Launch Time":1427397593820,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4073,"Index":65,"Attempt":0,"Launch Time":1427397593804,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593821,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1547586,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4083,"Index":75,"Attempt":0,"Launch Time":1427397593821,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4078,"Index":70,"Attempt":0,"Launch Time":1427397593808,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593822,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":288286,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4084,"Index":76,"Attempt":0,"Launch Time":1427397593826,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4080,"Index":72,"Attempt":0,"Launch Time":1427397593809,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593826,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":307021,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4085,"Index":77,"Attempt":0,"Launch Time":1427397593829,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4079,"Index":71,"Attempt":0,"Launch Time":1427397593809,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593829,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":19,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":422406,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4086,"Index":78,"Attempt":0,"Launch Time":1427397593830,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4083,"Index":75,"Attempt":0,"Launch Time":1427397593821,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593831,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":400339,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4087,"Index":79,"Attempt":0,"Launch Time":1427397593831,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4074,"Index":66,"Attempt":0,"Launch Time":1427397593806,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593831,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":23,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":362316,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4088,"Index":80,"Attempt":0,"Launch Time":1427397593832,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4077,"Index":69,"Attempt":0,"Launch Time":1427397593807,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593832,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":23,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":305782,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4089,"Index":81,"Attempt":0,"Launch Time":1427397593834,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4084,"Index":76,"Attempt":0,"Launch Time":1427397593826,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593834,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":651502,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4090,"Index":82,"Attempt":0,"Launch Time":1427397593834,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4082,"Index":74,"Attempt":0,"Launch Time":1427397593820,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593834,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":313038,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4091,"Index":83,"Attempt":0,"Launch Time":1427397593835,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4085,"Index":77,"Attempt":0,"Launch Time":1427397593829,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593835,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":305523,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4092,"Index":84,"Attempt":0,"Launch Time":1427397593837,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4086,"Index":78,"Attempt":0,"Launch Time":1427397593830,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593837,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":311004,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4093,"Index":85,"Attempt":0,"Launch Time":1427397593839,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4088,"Index":80,"Attempt":0,"Launch Time":1427397593832,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593839,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":303992,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4094,"Index":86,"Attempt":0,"Launch Time":1427397593841,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4090,"Index":82,"Attempt":0,"Launch Time":1427397593834,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593841,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":279207,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4095,"Index":87,"Attempt":0,"Launch Time":1427397593842,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4089,"Index":81,"Attempt":0,"Launch Time":1427397593834,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593842,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":304089,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4096,"Index":88,"Attempt":0,"Launch Time":1427397593843,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4092,"Index":84,"Attempt":0,"Launch Time":1427397593837,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593843,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":274823,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4097,"Index":89,"Attempt":0,"Launch Time":1427397593844,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4091,"Index":83,"Attempt":0,"Launch Time":1427397593835,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593844,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":272098,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4098,"Index":90,"Attempt":0,"Launch Time":1427397593846,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4093,"Index":85,"Attempt":0,"Launch Time":1427397593839,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593846,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":280126,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4099,"Index":91,"Attempt":0,"Launch Time":1427397593847,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4094,"Index":86,"Attempt":0,"Launch Time":1427397593841,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593847,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":263265,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4100,"Index":92,"Attempt":0,"Launch Time":1427397593849,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4096,"Index":88,"Attempt":0,"Launch Time":1427397593843,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593849,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":261761,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4101,"Index":93,"Attempt":0,"Launch Time":1427397593850,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4097,"Index":89,"Attempt":0,"Launch Time":1427397593844,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593850,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":274232,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4102,"Index":94,"Attempt":0,"Launch Time":1427397593850,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4095,"Index":87,"Attempt":0,"Launch Time":1427397593842,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593850,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":268298,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4103,"Index":95,"Attempt":0,"Launch Time":1427397593852,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4076,"Index":68,"Attempt":0,"Launch Time":1427397593806,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593852,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":45,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":518506,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4104,"Index":96,"Attempt":0,"Launch Time":1427397593853,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4081,"Index":73,"Attempt":0,"Launch Time":1427397593820,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593853,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":33,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1316831,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4105,"Index":97,"Attempt":0,"Launch Time":1427397593854,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4098,"Index":90,"Attempt":0,"Launch Time":1427397593846,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593854,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":328064,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4106,"Index":98,"Attempt":0,"Launch Time":1427397593854,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4099,"Index":91,"Attempt":0,"Launch Time":1427397593847,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593854,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":325023,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4107,"Index":99,"Attempt":0,"Launch Time":1427397593857,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4102,"Index":94,"Attempt":0,"Launch Time":1427397593850,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593857,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":317818,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4108,"Index":100,"Attempt":0,"Launch Time":1427397593857,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4100,"Index":92,"Attempt":0,"Launch Time":1427397593849,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593857,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":362768,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4109,"Index":101,"Attempt":0,"Launch Time":1427397593860,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4106,"Index":98,"Attempt":0,"Launch Time":1427397593854,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593860,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":250758,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4110,"Index":102,"Attempt":0,"Launch Time":1427397593861,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4103,"Index":95,"Attempt":0,"Launch Time":1427397593852,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593861,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":282720,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4111,"Index":103,"Attempt":0,"Launch Time":1427397593863,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4108,"Index":100,"Attempt":0,"Launch Time":1427397593857,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593863,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":273606,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4112,"Index":104,"Attempt":0,"Launch Time":1427397593864,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4101,"Index":93,"Attempt":0,"Launch Time":1427397593850,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593864,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":4773208,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4113,"Index":105,"Attempt":0,"Launch Time":1427397593865,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4109,"Index":101,"Attempt":0,"Launch Time":1427397593860,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593865,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":222257,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4114,"Index":106,"Attempt":0,"Launch Time":1427397593868,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4105,"Index":97,"Attempt":0,"Launch Time":1427397593854,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593868,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":220351,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4115,"Index":107,"Attempt":0,"Launch Time":1427397593870,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4111,"Index":103,"Attempt":0,"Launch Time":1427397593863,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593870,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":276426,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4116,"Index":108,"Attempt":0,"Launch Time":1427397593870,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4112,"Index":104,"Attempt":0,"Launch Time":1427397593864,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593870,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":273506,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4117,"Index":109,"Attempt":0,"Launch Time":1427397593871,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4113,"Index":105,"Attempt":0,"Launch Time":1427397593865,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593871,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":261141,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4118,"Index":110,"Attempt":0,"Launch Time":1427397593872,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4107,"Index":99,"Attempt":0,"Launch Time":1427397593857,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593873,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":365339,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4119,"Index":111,"Attempt":0,"Launch Time":1427397593874,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4114,"Index":106,"Attempt":0,"Launch Time":1427397593868,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593874,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":315977,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4120,"Index":112,"Attempt":0,"Launch Time":1427397593876,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4115,"Index":107,"Attempt":0,"Launch Time":1427397593870,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593876,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":273288,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4121,"Index":113,"Attempt":0,"Launch Time":1427397593877,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4116,"Index":108,"Attempt":0,"Launch Time":1427397593870,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593877,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":282320,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4122,"Index":114,"Attempt":0,"Launch Time":1427397593877,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4117,"Index":109,"Attempt":0,"Launch Time":1427397593871,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593877,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":267537,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4123,"Index":115,"Attempt":0,"Launch Time":1427397593882,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4119,"Index":111,"Attempt":0,"Launch Time":1427397593874,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593883,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1426520,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4124,"Index":116,"Attempt":0,"Launch Time":1427397593890,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4087,"Index":79,"Attempt":0,"Launch Time":1427397593831,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593890,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":11,"Executor Run Time":37,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":329699,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4125,"Index":117,"Attempt":0,"Launch Time":1427397593893,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4121,"Index":113,"Attempt":0,"Launch Time":1427397593877,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593893,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":345504,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4126,"Index":118,"Attempt":0,"Launch Time":1427397593894,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4104,"Index":96,"Attempt":0,"Launch Time":1427397593853,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593894,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":18,"Executor Run Time":19,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":434417,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4127,"Index":119,"Attempt":0,"Launch Time":1427397593896,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4118,"Index":110,"Attempt":0,"Launch Time":1427397593872,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593896,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":279988,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4128,"Index":120,"Attempt":0,"Launch Time":1427397593898,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4124,"Index":116,"Attempt":0,"Launch Time":1427397593890,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593899,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":318000,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4129,"Index":121,"Attempt":0,"Launch Time":1427397593902,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4126,"Index":118,"Attempt":0,"Launch Time":1427397593894,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593902,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":254380,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4130,"Index":122,"Attempt":0,"Launch Time":1427397593902,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4127,"Index":119,"Attempt":0,"Launch Time":1427397593896,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593902,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":267506,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4131,"Index":123,"Attempt":0,"Launch Time":1427397593907,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4125,"Index":117,"Attempt":0,"Launch Time":1427397593893,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593907,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":517869,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4132,"Index":124,"Attempt":0,"Launch Time":1427397593908,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4130,"Index":122,"Attempt":0,"Launch Time":1427397593902,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593908,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":291277,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4133,"Index":125,"Attempt":0,"Launch Time":1427397593910,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4128,"Index":120,"Attempt":0,"Launch Time":1427397593898,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593910,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":283559,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4134,"Index":126,"Attempt":0,"Launch Time":1427397593914,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4132,"Index":124,"Attempt":0,"Launch Time":1427397593908,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593914,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":285913,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4135,"Index":127,"Attempt":0,"Launch Time":1427397593916,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4131,"Index":123,"Attempt":0,"Launch Time":1427397593907,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593916,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":330322,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4136,"Index":128,"Attempt":0,"Launch Time":1427397593917,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4133,"Index":125,"Attempt":0,"Launch Time":1427397593910,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593917,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":272297,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4137,"Index":129,"Attempt":0,"Launch Time":1427397593921,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4134,"Index":126,"Attempt":0,"Launch Time":1427397593914,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593921,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":238008,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4138,"Index":130,"Attempt":0,"Launch Time":1427397593923,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4136,"Index":128,"Attempt":0,"Launch Time":1427397593917,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593923,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":264420,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4139,"Index":131,"Attempt":0,"Launch Time":1427397593924,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4135,"Index":127,"Attempt":0,"Launch Time":1427397593916,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593924,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":269186,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4140,"Index":132,"Attempt":0,"Launch Time":1427397593927,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4137,"Index":129,"Attempt":0,"Launch Time":1427397593921,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593927,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":245361,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4141,"Index":133,"Attempt":0,"Launch Time":1427397593930,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4139,"Index":131,"Attempt":0,"Launch Time":1427397593924,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593930,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":284209,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4142,"Index":134,"Attempt":0,"Launch Time":1427397593931,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4138,"Index":130,"Attempt":0,"Launch Time":1427397593923,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593931,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":255791,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4143,"Index":135,"Attempt":0,"Launch Time":1427397593932,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4122,"Index":114,"Attempt":0,"Launch Time":1427397593877,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593932,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":54,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":294412,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4144,"Index":136,"Attempt":0,"Launch Time":1427397593933,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4140,"Index":132,"Attempt":0,"Launch Time":1427397593927,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593933,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":331435,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4145,"Index":137,"Attempt":0,"Launch Time":1427397593935,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4120,"Index":112,"Attempt":0,"Launch Time":1427397593876,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593936,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":54,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":290161,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4146,"Index":138,"Attempt":0,"Launch Time":1427397593937,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4123,"Index":115,"Attempt":0,"Launch Time":1427397593882,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593937,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":53,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":346710,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4147,"Index":139,"Attempt":0,"Launch Time":1427397593937,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4142,"Index":134,"Attempt":0,"Launch Time":1427397593931,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593937,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":324455,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4148,"Index":140,"Attempt":0,"Launch Time":1427397593940,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4141,"Index":133,"Attempt":0,"Launch Time":1427397593930,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593940,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":338934,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4149,"Index":141,"Attempt":0,"Launch Time":1427397593940,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4144,"Index":136,"Attempt":0,"Launch Time":1427397593933,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593940,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":314267,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4150,"Index":142,"Attempt":0,"Launch Time":1427397593942,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4110,"Index":102,"Attempt":0,"Launch Time":1427397593861,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593942,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":60,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":431493,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4151,"Index":143,"Attempt":0,"Launch Time":1427397593944,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4146,"Index":138,"Attempt":0,"Launch Time":1427397593937,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593944,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":263845,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4152,"Index":144,"Attempt":0,"Launch Time":1427397593946,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4148,"Index":140,"Attempt":0,"Launch Time":1427397593940,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593946,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9606,"Shuffle Write Time":256320,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4153,"Index":145,"Attempt":0,"Launch Time":1427397593948,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4150,"Index":142,"Attempt":0,"Launch Time":1427397593942,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593948,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":297662,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4154,"Index":146,"Attempt":0,"Launch Time":1427397593950,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4151,"Index":143,"Attempt":0,"Launch Time":1427397593944,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593950,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":298505,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4155,"Index":147,"Attempt":0,"Launch Time":1427397593950,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4147,"Index":139,"Attempt":0,"Launch Time":1427397593937,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593950,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":268119,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4156,"Index":148,"Attempt":0,"Launch Time":1427397593952,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4157,"Index":149,"Attempt":0,"Launch Time":1427397593952,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4143,"Index":135,"Attempt":0,"Launch Time":1427397593932,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593952,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":20,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":304088,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4149,"Index":141,"Attempt":0,"Launch Time":1427397593940,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593952,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":292971,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4158,"Index":150,"Attempt":0,"Launch Time":1427397593955,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4154,"Index":146,"Attempt":0,"Launch Time":1427397593950,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593956,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":284204,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4159,"Index":151,"Attempt":0,"Launch Time":1427397593956,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4152,"Index":144,"Attempt":0,"Launch Time":1427397593946,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593956,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":327743,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4160,"Index":152,"Attempt":0,"Launch Time":1427397593957,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4153,"Index":145,"Attempt":0,"Launch Time":1427397593948,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593957,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":259864,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4161,"Index":153,"Attempt":0,"Launch Time":1427397593959,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4157,"Index":149,"Attempt":0,"Launch Time":1427397593952,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593959,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":264828,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4162,"Index":154,"Attempt":0,"Launch Time":1427397593962,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4145,"Index":137,"Attempt":0,"Launch Time":1427397593935,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593962,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":242789,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4158,"Index":150,"Attempt":0,"Launch Time":1427397593955,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593963,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":268649,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4163,"Index":155,"Attempt":0,"Launch Time":1427397593963,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4164,"Index":156,"Attempt":0,"Launch Time":1427397593965,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4159,"Index":151,"Attempt":0,"Launch Time":1427397593956,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593965,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":282955,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4165,"Index":157,"Attempt":0,"Launch Time":1427397593966,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4161,"Index":153,"Attempt":0,"Launch Time":1427397593959,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593966,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":282858,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4166,"Index":158,"Attempt":0,"Launch Time":1427397593968,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4162,"Index":154,"Attempt":0,"Launch Time":1427397593962,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593968,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":280732,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4167,"Index":159,"Attempt":0,"Launch Time":1427397593969,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4160,"Index":152,"Attempt":0,"Launch Time":1427397593957,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593970,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":328308,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4168,"Index":160,"Attempt":0,"Launch Time":1427397593971,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4155,"Index":147,"Attempt":0,"Launch Time":1427397593950,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593971,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":20,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":3026501,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4169,"Index":161,"Attempt":0,"Launch Time":1427397593973,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4165,"Index":157,"Attempt":0,"Launch Time":1427397593966,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593979,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":392185,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4170,"Index":162,"Attempt":0,"Launch Time":1427397593980,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4166,"Index":158,"Attempt":0,"Launch Time":1427397593968,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593980,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":5917945,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4171,"Index":163,"Attempt":0,"Launch Time":1427397593981,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4156,"Index":148,"Attempt":0,"Launch Time":1427397593952,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593981,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":299216,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4172,"Index":164,"Attempt":0,"Launch Time":1427397593983,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4164,"Index":156,"Attempt":0,"Launch Time":1427397593965,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593983,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":16,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":370623,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4173,"Index":165,"Attempt":0,"Launch Time":1427397593983,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4167,"Index":159,"Attempt":0,"Launch Time":1427397593969,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593983,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":276415,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4174,"Index":166,"Attempt":0,"Launch Time":1427397593986,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4175,"Index":167,"Attempt":0,"Launch Time":1427397593986,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4170,"Index":162,"Attempt":0,"Launch Time":1427397593980,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593986,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":254114,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4169,"Index":161,"Attempt":0,"Launch Time":1427397593973,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593986,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":294943,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4176,"Index":168,"Attempt":0,"Launch Time":1427397593988,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4168,"Index":160,"Attempt":0,"Launch Time":1427397593971,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593988,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":15,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":451476,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4177,"Index":169,"Attempt":0,"Launch Time":1427397593989,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4173,"Index":165,"Attempt":0,"Launch Time":1427397593983,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593989,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":274036,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4178,"Index":170,"Attempt":0,"Launch Time":1427397593989,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4171,"Index":163,"Attempt":0,"Launch Time":1427397593981,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593989,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":275731,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4179,"Index":171,"Attempt":0,"Launch Time":1427397593991,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4180,"Index":172,"Attempt":0,"Launch Time":1427397593992,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4181,"Index":173,"Attempt":0,"Launch Time":1427397593992,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4172,"Index":164,"Attempt":0,"Launch Time":1427397593983,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593992,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":288918,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4174,"Index":166,"Attempt":0,"Launch Time":1427397593986,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593992,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":238619,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4175,"Index":167,"Attempt":0,"Launch Time":1427397593986,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593992,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":260413,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4182,"Index":174,"Attempt":0,"Launch Time":1427397593997,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4176,"Index":168,"Attempt":0,"Launch Time":1427397593988,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593997,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":370589,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4183,"Index":175,"Attempt":0,"Launch Time":1427397593997,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4163,"Index":155,"Attempt":0,"Launch Time":1427397593963,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593997,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":34,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":281915,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4184,"Index":176,"Attempt":0,"Launch Time":1427397593997,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4179,"Index":171,"Attempt":0,"Launch Time":1427397593991,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593997,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":271126,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4185,"Index":177,"Attempt":0,"Launch Time":1427397593998,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4180,"Index":172,"Attempt":0,"Launch Time":1427397593992,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593998,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":326081,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4186,"Index":178,"Attempt":0,"Launch Time":1427397593999,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4187,"Index":179,"Attempt":0,"Launch Time":1427397593999,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4177,"Index":169,"Attempt":0,"Launch Time":1427397593989,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593999,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":569415,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4181,"Index":173,"Attempt":0,"Launch Time":1427397593992,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593999,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":282128,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4188,"Index":180,"Attempt":0,"Launch Time":1427397594003,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4184,"Index":176,"Attempt":0,"Launch Time":1427397593997,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594003,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":257720,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4189,"Index":181,"Attempt":0,"Launch Time":1427397594004,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4185,"Index":177,"Attempt":0,"Launch Time":1427397593998,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594004,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":254218,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4190,"Index":182,"Attempt":0,"Launch Time":1427397594006,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4186,"Index":178,"Attempt":0,"Launch Time":1427397593999,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594006,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":251967,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4191,"Index":183,"Attempt":0,"Launch Time":1427397594008,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4188,"Index":180,"Attempt":0,"Launch Time":1427397594003,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594009,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":276343,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4192,"Index":184,"Attempt":0,"Launch Time":1427397594009,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4189,"Index":181,"Attempt":0,"Launch Time":1427397594004,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594009,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":247846,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4193,"Index":185,"Attempt":0,"Launch Time":1427397594011,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4187,"Index":179,"Attempt":0,"Launch Time":1427397593999,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594011,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":232146,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4194,"Index":186,"Attempt":0,"Launch Time":1427397594012,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4190,"Index":182,"Attempt":0,"Launch Time":1427397594006,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594012,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":265430,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4195,"Index":187,"Attempt":0,"Launch Time":1427397594012,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4182,"Index":174,"Attempt":0,"Launch Time":1427397593997,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594012,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":263078,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4196,"Index":188,"Attempt":0,"Launch Time":1427397594015,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4178,"Index":170,"Attempt":0,"Launch Time":1427397593989,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594015,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":25,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":315828,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4197,"Index":189,"Attempt":0,"Launch Time":1427397594017,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4192,"Index":184,"Attempt":0,"Launch Time":1427397594009,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594017,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1530912,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4198,"Index":190,"Attempt":0,"Launch Time":1427397594018,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4193,"Index":185,"Attempt":0,"Launch Time":1427397594011,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594019,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":368859,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4199,"Index":191,"Attempt":0,"Launch Time":1427397594019,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4129,"Index":121,"Attempt":0,"Launch Time":1427397593902,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594019,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":116,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":4062514,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4200,"Index":192,"Attempt":0,"Launch Time":1427397594020,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4183,"Index":175,"Attempt":0,"Launch Time":1427397593997,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594020,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":348428,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4201,"Index":193,"Attempt":0,"Launch Time":1427397594022,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4196,"Index":188,"Attempt":0,"Launch Time":1427397594015,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594022,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":354798,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4202,"Index":194,"Attempt":0,"Launch Time":1427397594025,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4197,"Index":189,"Attempt":0,"Launch Time":1427397594017,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594025,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":329059,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4203,"Index":195,"Attempt":0,"Launch Time":1427397594027,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4204,"Index":196,"Attempt":0,"Launch Time":1427397594027,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4198,"Index":190,"Attempt":0,"Launch Time":1427397594018,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594027,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":261897,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4199,"Index":191,"Attempt":0,"Launch Time":1427397594019,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594027,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":268083,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4205,"Index":197,"Attempt":0,"Launch Time":1427397594028,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4201,"Index":193,"Attempt":0,"Launch Time":1427397594022,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594028,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":291222,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4206,"Index":198,"Attempt":0,"Launch Time":1427397594033,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4200,"Index":192,"Attempt":0,"Launch Time":1427397594020,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594033,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":289506,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4202,"Index":194,"Attempt":0,"Launch Time":1427397594025,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594033,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":292904,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4207,"Index":199,"Attempt":0,"Launch Time":1427397594033,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4208,"Index":200,"Attempt":0,"Launch Time":1427397594034,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4204,"Index":196,"Attempt":0,"Launch Time":1427397594027,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594034,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":451595,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4209,"Index":201,"Attempt":0,"Launch Time":1427397594035,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4205,"Index":197,"Attempt":0,"Launch Time":1427397594028,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594035,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":296983,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4210,"Index":202,"Attempt":0,"Launch Time":1427397594039,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4207,"Index":199,"Attempt":0,"Launch Time":1427397594033,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594039,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":265124,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4211,"Index":203,"Attempt":0,"Launch Time":1427397594040,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4208,"Index":200,"Attempt":0,"Launch Time":1427397594034,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594040,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":274641,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4212,"Index":204,"Attempt":0,"Launch Time":1427397594041,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4209,"Index":201,"Attempt":0,"Launch Time":1427397594035,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594041,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":262055,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4213,"Index":205,"Attempt":0,"Launch Time":1427397594044,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4203,"Index":195,"Attempt":0,"Launch Time":1427397594027,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594044,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":539141,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4214,"Index":206,"Attempt":0,"Launch Time":1427397594046,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4211,"Index":203,"Attempt":0,"Launch Time":1427397594040,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594046,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":308156,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4215,"Index":207,"Attempt":0,"Launch Time":1427397594053,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4210,"Index":202,"Attempt":0,"Launch Time":1427397594039,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594053,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":320607,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4216,"Index":208,"Attempt":0,"Launch Time":1427397594057,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4206,"Index":198,"Attempt":0,"Launch Time":1427397594033,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594057,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":22,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":559993,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4217,"Index":209,"Attempt":0,"Launch Time":1427397594057,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4213,"Index":205,"Attempt":0,"Launch Time":1427397594044,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594057,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":269069,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4218,"Index":210,"Attempt":0,"Launch Time":1427397594057,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4212,"Index":204,"Attempt":0,"Launch Time":1427397594041,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594057,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":280616,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4219,"Index":211,"Attempt":0,"Launch Time":1427397594059,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4215,"Index":207,"Attempt":0,"Launch Time":1427397594053,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594059,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":238848,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4220,"Index":212,"Attempt":0,"Launch Time":1427397594061,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4214,"Index":206,"Attempt":0,"Launch Time":1427397594046,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594061,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":323953,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4221,"Index":213,"Attempt":0,"Launch Time":1427397594064,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4218,"Index":210,"Attempt":0,"Launch Time":1427397594057,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594064,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":270239,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4222,"Index":214,"Attempt":0,"Launch Time":1427397594064,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4219,"Index":211,"Attempt":0,"Launch Time":1427397594059,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594064,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":277776,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4223,"Index":215,"Attempt":0,"Launch Time":1427397594072,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4222,"Index":214,"Attempt":0,"Launch Time":1427397594064,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594072,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":324732,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4224,"Index":216,"Attempt":0,"Launch Time":1427397594072,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4220,"Index":212,"Attempt":0,"Launch Time":1427397594061,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594072,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":368112,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4225,"Index":217,"Attempt":0,"Launch Time":1427397594073,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4216,"Index":208,"Attempt":0,"Launch Time":1427397594057,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594073,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":366276,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4226,"Index":218,"Attempt":0,"Launch Time":1427397594076,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4217,"Index":209,"Attempt":0,"Launch Time":1427397594057,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594076,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":319383,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4227,"Index":219,"Attempt":0,"Launch Time":1427397594076,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4221,"Index":213,"Attempt":0,"Launch Time":1427397594064,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594077,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":309587,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4228,"Index":220,"Attempt":0,"Launch Time":1427397594078,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4224,"Index":216,"Attempt":0,"Launch Time":1427397594072,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594078,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":258865,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4229,"Index":221,"Attempt":0,"Launch Time":1427397594078,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4223,"Index":215,"Attempt":0,"Launch Time":1427397594072,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594078,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":283494,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4230,"Index":222,"Attempt":0,"Launch Time":1427397594078,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4225,"Index":217,"Attempt":0,"Launch Time":1427397594073,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594078,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":273191,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4231,"Index":223,"Attempt":0,"Launch Time":1427397594085,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4195,"Index":187,"Attempt":0,"Launch Time":1427397594012,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594085,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":21,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":310401,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4232,"Index":224,"Attempt":0,"Launch Time":1427397594085,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4233,"Index":225,"Attempt":0,"Launch Time":1427397594085,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4230,"Index":222,"Attempt":0,"Launch Time":1427397594078,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594085,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":296414,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4229,"Index":221,"Attempt":0,"Launch Time":1427397594078,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594086,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":299618,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4234,"Index":226,"Attempt":0,"Launch Time":1427397594087,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4228,"Index":220,"Attempt":0,"Launch Time":1427397594078,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594087,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":295606,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4235,"Index":227,"Attempt":0,"Launch Time":1427397594087,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4227,"Index":219,"Attempt":0,"Launch Time":1427397594076,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594087,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":272301,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4236,"Index":228,"Attempt":0,"Launch Time":1427397594091,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4231,"Index":223,"Attempt":0,"Launch Time":1427397594085,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594091,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":256734,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4237,"Index":229,"Attempt":0,"Launch Time":1427397594091,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4232,"Index":224,"Attempt":0,"Launch Time":1427397594085,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594091,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":273003,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4238,"Index":230,"Attempt":0,"Launch Time":1427397594092,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4234,"Index":226,"Attempt":0,"Launch Time":1427397594087,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594092,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":265995,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4239,"Index":231,"Attempt":0,"Launch Time":1427397594093,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4235,"Index":227,"Attempt":0,"Launch Time":1427397594087,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594093,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":253134,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4240,"Index":232,"Attempt":0,"Launch Time":1427397594096,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4236,"Index":228,"Attempt":0,"Launch Time":1427397594091,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594096,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":250411,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4241,"Index":233,"Attempt":0,"Launch Time":1427397594099,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4237,"Index":229,"Attempt":0,"Launch Time":1427397594091,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594099,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":351852,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4242,"Index":234,"Attempt":0,"Launch Time":1427397594102,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4238,"Index":230,"Attempt":0,"Launch Time":1427397594092,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594102,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":276492,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4243,"Index":235,"Attempt":0,"Launch Time":1427397594102,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4240,"Index":232,"Attempt":0,"Launch Time":1427397594096,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594102,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":243302,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4244,"Index":236,"Attempt":0,"Launch Time":1427397594106,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4241,"Index":233,"Attempt":0,"Launch Time":1427397594099,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594106,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":282740,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4245,"Index":237,"Attempt":0,"Launch Time":1427397594107,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4242,"Index":234,"Attempt":0,"Launch Time":1427397594102,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594107,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":273630,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4246,"Index":238,"Attempt":0,"Launch Time":1427397594108,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4243,"Index":235,"Attempt":0,"Launch Time":1427397594102,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594108,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":274026,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4247,"Index":239,"Attempt":0,"Launch Time":1427397594112,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4244,"Index":236,"Attempt":0,"Launch Time":1427397594106,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594112,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":322557,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4248,"Index":240,"Attempt":0,"Launch Time":1427397594114,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4239,"Index":231,"Attempt":0,"Launch Time":1427397594093,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594114,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":20,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1976562,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4249,"Index":241,"Attempt":0,"Launch Time":1427397594114,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4246,"Index":238,"Attempt":0,"Launch Time":1427397594108,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594114,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":299755,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4250,"Index":242,"Attempt":0,"Launch Time":1427397594116,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4245,"Index":237,"Attempt":0,"Launch Time":1427397594107,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594116,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":509637,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4251,"Index":243,"Attempt":0,"Launch Time":1427397594119,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4247,"Index":239,"Attempt":0,"Launch Time":1427397594112,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594120,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":367314,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4252,"Index":244,"Attempt":0,"Launch Time":1427397594120,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4249,"Index":241,"Attempt":0,"Launch Time":1427397594114,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594120,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":319305,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4253,"Index":245,"Attempt":0,"Launch Time":1427397594126,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4250,"Index":242,"Attempt":0,"Launch Time":1427397594116,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594126,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":361154,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4254,"Index":246,"Attempt":0,"Launch Time":1427397594127,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4251,"Index":243,"Attempt":0,"Launch Time":1427397594119,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594127,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":272810,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4255,"Index":247,"Attempt":0,"Launch Time":1427397594128,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4252,"Index":244,"Attempt":0,"Launch Time":1427397594120,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594128,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":280081,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4256,"Index":248,"Attempt":0,"Launch Time":1427397594129,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4226,"Index":218,"Attempt":0,"Launch Time":1427397594076,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594129,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":50,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":442113,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4257,"Index":249,"Attempt":0,"Launch Time":1427397594131,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4248,"Index":240,"Attempt":0,"Launch Time":1427397594114,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594131,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":498765,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4258,"Index":250,"Attempt":0,"Launch Time":1427397594132,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4253,"Index":245,"Attempt":0,"Launch Time":1427397594126,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594132,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":273227,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4259,"Index":251,"Attempt":0,"Launch Time":1427397594132,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4254,"Index":246,"Attempt":0,"Launch Time":1427397594127,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594132,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":264033,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4260,"Index":252,"Attempt":0,"Launch Time":1427397594134,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4255,"Index":247,"Attempt":0,"Launch Time":1427397594128,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594134,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":265565,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4261,"Index":253,"Attempt":0,"Launch Time":1427397594145,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594152,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4191,"Index":183,"Attempt":0,"Launch Time":1427397594008,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594145,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":136,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":283942,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4262,"Index":254,"Attempt":0,"Launch Time":1427397594146,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594153,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4233,"Index":225,"Attempt":0,"Launch Time":1427397594085,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594146,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":28,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":301188,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4263,"Index":255,"Attempt":0,"Launch Time":1427397594148,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594155,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4194,"Index":186,"Attempt":0,"Launch Time":1427397594012,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594148,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":88,"Executor Run Time":6,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":273456,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4264,"Index":256,"Attempt":0,"Launch Time":1427397594152,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594159,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4261,"Index":253,"Attempt":0,"Launch Time":1427397594145,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594152,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":340529,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4265,"Index":257,"Attempt":0,"Launch Time":1427397594152,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594159,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4262,"Index":254,"Attempt":0,"Launch Time":1427397594146,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594153,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":361681,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4266,"Index":258,"Attempt":0,"Launch Time":1427397594155,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594161,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4263,"Index":255,"Attempt":0,"Launch Time":1427397594148,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594155,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":324964,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4267,"Index":259,"Attempt":0,"Launch Time":1427397594159,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594165,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4264,"Index":256,"Attempt":0,"Launch Time":1427397594152,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594159,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":289776,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4268,"Index":260,"Attempt":0,"Launch Time":1427397594159,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594166,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4265,"Index":257,"Attempt":0,"Launch Time":1427397594152,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594159,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9613,"Shuffle Write Time":302377,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4269,"Index":261,"Attempt":0,"Launch Time":1427397594161,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594167,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4266,"Index":258,"Attempt":0,"Launch Time":1427397594155,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594161,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":626399,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4270,"Index":262,"Attempt":0,"Launch Time":1427397594165,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594173,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4267,"Index":259,"Attempt":0,"Launch Time":1427397594159,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594165,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":283515,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4271,"Index":263,"Attempt":0,"Launch Time":1427397594166,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594172,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4268,"Index":260,"Attempt":0,"Launch Time":1427397594159,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594166,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":288842,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4272,"Index":264,"Attempt":0,"Launch Time":1427397594167,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594174,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4269,"Index":261,"Attempt":0,"Launch Time":1427397594161,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594167,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":277967,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4273,"Index":265,"Attempt":0,"Launch Time":1427397594172,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594178,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4257,"Index":249,"Attempt":0,"Launch Time":1427397594131,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594172,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":41,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9613,"Shuffle Write Time":304628,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4274,"Index":266,"Attempt":0,"Launch Time":1427397594172,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594178,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4271,"Index":263,"Attempt":0,"Launch Time":1427397594166,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594172,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":285914,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4275,"Index":267,"Attempt":0,"Launch Time":1427397594173,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594180,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4270,"Index":262,"Attempt":0,"Launch Time":1427397594165,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594173,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":470938,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4276,"Index":268,"Attempt":0,"Launch Time":1427397594174,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594180,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4272,"Index":264,"Attempt":0,"Launch Time":1427397594167,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594174,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":854969,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4277,"Index":269,"Attempt":0,"Launch Time":1427397594178,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594191,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4273,"Index":265,"Attempt":0,"Launch Time":1427397594172,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594178,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9613,"Shuffle Write Time":248967,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4278,"Index":270,"Attempt":0,"Launch Time":1427397594178,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594184,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4274,"Index":266,"Attempt":0,"Launch Time":1427397594172,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594178,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":272225,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4279,"Index":271,"Attempt":0,"Launch Time":1427397594180,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594186,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4280,"Index":272,"Attempt":0,"Launch Time":1427397594180,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594191,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4276,"Index":268,"Attempt":0,"Launch Time":1427397594174,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594180,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":263445,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4275,"Index":267,"Attempt":0,"Launch Time":1427397594173,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594180,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":269248,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4281,"Index":273,"Attempt":0,"Launch Time":1427397594183,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594189,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4282,"Index":274,"Attempt":0,"Launch Time":1427397594184,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594190,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4278,"Index":270,"Attempt":0,"Launch Time":1427397594178,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594184,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":286626,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4283,"Index":275,"Attempt":0,"Launch Time":1427397594186,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594192,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4279,"Index":271,"Attempt":0,"Launch Time":1427397594180,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594186,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":278118,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4284,"Index":276,"Attempt":0,"Launch Time":1427397594189,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594195,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4281,"Index":273,"Attempt":0,"Launch Time":1427397594183,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594189,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":259293,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4285,"Index":277,"Attempt":0,"Launch Time":1427397594190,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594196,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4282,"Index":274,"Attempt":0,"Launch Time":1427397594184,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594190,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":264635,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4286,"Index":278,"Attempt":0,"Launch Time":1427397594190,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594197,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4280,"Index":272,"Attempt":0,"Launch Time":1427397594180,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594191,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":262005,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4277,"Index":269,"Attempt":0,"Launch Time":1427397594178,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594191,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":250216,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4287,"Index":279,"Attempt":0,"Launch Time":1427397594192,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594198,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4283,"Index":275,"Attempt":0,"Launch Time":1427397594186,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594192,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":268695,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4288,"Index":280,"Attempt":0,"Launch Time":1427397594195,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594203,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4284,"Index":276,"Attempt":0,"Launch Time":1427397594189,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594195,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":273333,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4289,"Index":281,"Attempt":0,"Launch Time":1427397594196,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594203,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4285,"Index":277,"Attempt":0,"Launch Time":1427397594190,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594196,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":280310,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4290,"Index":282,"Attempt":0,"Launch Time":1427397594197,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594206,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4286,"Index":278,"Attempt":0,"Launch Time":1427397594190,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594197,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":263164,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4291,"Index":283,"Attempt":0,"Launch Time":1427397594198,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594207,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4287,"Index":279,"Attempt":0,"Launch Time":1427397594192,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594198,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":290598,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4292,"Index":284,"Attempt":0,"Launch Time":1427397594203,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594209,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4288,"Index":280,"Attempt":0,"Launch Time":1427397594195,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594203,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":371191,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4293,"Index":285,"Attempt":0,"Launch Time":1427397594203,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594209,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4289,"Index":281,"Attempt":0,"Launch Time":1427397594196,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594203,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":341649,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4294,"Index":286,"Attempt":0,"Launch Time":1427397594206,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594216,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4290,"Index":282,"Attempt":0,"Launch Time":1427397594197,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594206,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":394189,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4295,"Index":287,"Attempt":0,"Launch Time":1427397594207,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594214,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4291,"Index":283,"Attempt":0,"Launch Time":1427397594198,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594207,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":397478,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4296,"Index":288,"Attempt":0,"Launch Time":1427397594209,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594215,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4292,"Index":284,"Attempt":0,"Launch Time":1427397594203,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594209,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":312928,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4297,"Index":289,"Attempt":0,"Launch Time":1427397594209,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594215,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4293,"Index":285,"Attempt":0,"Launch Time":1427397594203,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594209,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":280989,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4298,"Index":290,"Attempt":0,"Launch Time":1427397594214,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594222,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4295,"Index":287,"Attempt":0,"Launch Time":1427397594207,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594214,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":260288,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4299,"Index":291,"Attempt":0,"Launch Time":1427397594215,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594221,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4296,"Index":288,"Attempt":0,"Launch Time":1427397594209,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594215,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":287251,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4300,"Index":292,"Attempt":0,"Launch Time":1427397594215,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594221,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4297,"Index":289,"Attempt":0,"Launch Time":1427397594209,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594215,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":275462,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4301,"Index":293,"Attempt":0,"Launch Time":1427397594216,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594222,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4294,"Index":286,"Attempt":0,"Launch Time":1427397594206,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594216,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":294790,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4302,"Index":294,"Attempt":0,"Launch Time":1427397594221,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4303,"Index":295,"Attempt":0,"Launch Time":1427397594221,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4304,"Index":296,"Attempt":0,"Launch Time":1427397594221,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4300,"Index":292,"Attempt":0,"Launch Time":1427397594215,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594221,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":280740,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4299,"Index":291,"Attempt":0,"Launch Time":1427397594215,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594221,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":272038,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4298,"Index":290,"Attempt":0,"Launch Time":1427397594214,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594222,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":282546,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4305,"Index":297,"Attempt":0,"Launch Time":1427397594222,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594233,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4301,"Index":293,"Attempt":0,"Launch Time":1427397594216,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594222,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":291819,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4306,"Index":298,"Attempt":0,"Launch Time":1427397594233,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4305,"Index":297,"Attempt":0,"Launch Time":1427397594222,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594233,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":292253,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4307,"Index":299,"Attempt":0,"Launch Time":1427397594236,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4256,"Index":248,"Attempt":0,"Launch Time":1427397594129,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594236,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":106,"Result Size":930,"JVM GC Time":8,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":293072,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4308,"Index":300,"Attempt":0,"Launch Time":1427397594239,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4306,"Index":298,"Attempt":0,"Launch Time":1427397594233,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594239,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":279735,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4309,"Index":301,"Attempt":0,"Launch Time":1427397594242,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4307,"Index":299,"Attempt":0,"Launch Time":1427397594236,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594243,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":300366,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4310,"Index":302,"Attempt":0,"Launch Time":1427397594245,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4308,"Index":300,"Attempt":0,"Launch Time":1427397594239,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594245,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":273268,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4311,"Index":303,"Attempt":0,"Launch Time":1427397594246,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4303,"Index":295,"Attempt":0,"Launch Time":1427397594221,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594246,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":24,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":405418,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4312,"Index":304,"Attempt":0,"Launch Time":1427397594248,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4304,"Index":296,"Attempt":0,"Launch Time":1427397594221,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594248,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":27,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":290552,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4313,"Index":305,"Attempt":0,"Launch Time":1427397594248,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4302,"Index":294,"Attempt":0,"Launch Time":1427397594221,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594249,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":27,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":289857,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4314,"Index":306,"Attempt":0,"Launch Time":1427397594249,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4309,"Index":301,"Attempt":0,"Launch Time":1427397594242,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594249,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":307436,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4315,"Index":307,"Attempt":0,"Launch Time":1427397594254,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4311,"Index":303,"Attempt":0,"Launch Time":1427397594246,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594254,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":354308,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4316,"Index":308,"Attempt":0,"Launch Time":1427397594256,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4317,"Index":309,"Attempt":0,"Launch Time":1427397594256,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4310,"Index":302,"Attempt":0,"Launch Time":1427397594245,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594256,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":318426,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4313,"Index":305,"Attempt":0,"Launch Time":1427397594248,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594256,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":309779,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4318,"Index":310,"Attempt":0,"Launch Time":1427397594257,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4312,"Index":304,"Attempt":0,"Launch Time":1427397594248,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594257,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":315723,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4319,"Index":311,"Attempt":0,"Launch Time":1427397594261,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4315,"Index":307,"Attempt":0,"Launch Time":1427397594254,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594261,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":280814,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4320,"Index":312,"Attempt":0,"Launch Time":1427397594262,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4314,"Index":306,"Attempt":0,"Launch Time":1427397594249,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594262,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":344720,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4321,"Index":313,"Attempt":0,"Launch Time":1427397594262,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4317,"Index":309,"Attempt":0,"Launch Time":1427397594256,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594262,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":295462,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4322,"Index":314,"Attempt":0,"Launch Time":1427397594269,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4318,"Index":310,"Attempt":0,"Launch Time":1427397594257,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594269,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":284250,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4323,"Index":315,"Attempt":0,"Launch Time":1427397594269,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4324,"Index":316,"Attempt":0,"Launch Time":1427397594269,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4259,"Index":251,"Attempt":0,"Launch Time":1427397594132,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594269,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":133,"Result Size":930,"JVM GC Time":8,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":2000241,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4258,"Index":250,"Attempt":0,"Launch Time":1427397594132,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594270,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":49,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":270452,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4325,"Index":317,"Attempt":0,"Launch Time":1427397594270,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4326,"Index":318,"Attempt":0,"Launch Time":1427397594270,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4320,"Index":312,"Attempt":0,"Launch Time":1427397594262,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594270,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":297380,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4260,"Index":252,"Attempt":0,"Launch Time":1427397594134,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594270,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":50,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":816869,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4327,"Index":319,"Attempt":0,"Launch Time":1427397594270,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4321,"Index":313,"Attempt":0,"Launch Time":1427397594262,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594270,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":288200,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4328,"Index":320,"Attempt":0,"Launch Time":1427397594271,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4316,"Index":308,"Attempt":0,"Launch Time":1427397594256,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594271,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":275349,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4329,"Index":321,"Attempt":0,"Launch Time":1427397594277,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4328,"Index":320,"Attempt":0,"Launch Time":1427397594271,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594277,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":289377,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4330,"Index":322,"Attempt":0,"Launch Time":1427397594279,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4324,"Index":316,"Attempt":0,"Launch Time":1427397594269,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594279,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":269699,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4331,"Index":323,"Attempt":0,"Launch Time":1427397594279,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4325,"Index":317,"Attempt":0,"Launch Time":1427397594270,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594279,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":287203,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4332,"Index":324,"Attempt":0,"Launch Time":1427397594281,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4326,"Index":318,"Attempt":0,"Launch Time":1427397594270,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594281,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":294175,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4333,"Index":325,"Attempt":0,"Launch Time":1427397594282,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4329,"Index":321,"Attempt":0,"Launch Time":1427397594277,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594282,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":264391,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4334,"Index":326,"Attempt":0,"Launch Time":1427397594286,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4330,"Index":322,"Attempt":0,"Launch Time":1427397594279,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594286,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":263311,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4335,"Index":327,"Attempt":0,"Launch Time":1427397594287,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4331,"Index":323,"Attempt":0,"Launch Time":1427397594279,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594287,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":298025,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4336,"Index":328,"Attempt":0,"Launch Time":1427397594288,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4333,"Index":325,"Attempt":0,"Launch Time":1427397594282,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594288,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":233522,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4337,"Index":329,"Attempt":0,"Launch Time":1427397594292,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4327,"Index":319,"Attempt":0,"Launch Time":1427397594270,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594293,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":5,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":277988,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4338,"Index":330,"Attempt":0,"Launch Time":1427397594293,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4335,"Index":327,"Attempt":0,"Launch Time":1427397594287,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594293,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":263660,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4339,"Index":331,"Attempt":0,"Launch Time":1427397594293,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4332,"Index":324,"Attempt":0,"Launch Time":1427397594281,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594294,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":276259,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4340,"Index":332,"Attempt":0,"Launch Time":1427397594294,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4341,"Index":333,"Attempt":0,"Launch Time":1427397594295,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4336,"Index":328,"Attempt":0,"Launch Time":1427397594288,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594295,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":278660,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4334,"Index":326,"Attempt":0,"Launch Time":1427397594286,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594295,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":293194,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4338,"Index":330,"Attempt":0,"Launch Time":1427397594293,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594299,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":296931,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4342,"Index":334,"Attempt":0,"Launch Time":1427397594300,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4343,"Index":335,"Attempt":0,"Launch Time":1427397594302,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4340,"Index":332,"Attempt":0,"Launch Time":1427397594294,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594302,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":318899,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4344,"Index":336,"Attempt":0,"Launch Time":1427397594302,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4341,"Index":333,"Attempt":0,"Launch Time":1427397594295,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594303,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":342783,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4345,"Index":337,"Attempt":0,"Launch Time":1427397594303,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4339,"Index":331,"Attempt":0,"Launch Time":1427397594293,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594304,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":704728,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4346,"Index":338,"Attempt":0,"Launch Time":1427397594304,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4337,"Index":329,"Attempt":0,"Launch Time":1427397594292,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594304,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":421856,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4347,"Index":339,"Attempt":0,"Launch Time":1427397594307,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4342,"Index":334,"Attempt":0,"Launch Time":1427397594300,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594307,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9666,"Shuffle Write Time":383361,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4348,"Index":340,"Attempt":0,"Launch Time":1427397594309,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4349,"Index":341,"Attempt":0,"Launch Time":1427397594309,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4344,"Index":336,"Attempt":0,"Launch Time":1427397594302,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594309,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9648,"Shuffle Write Time":262787,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4343,"Index":335,"Attempt":0,"Launch Time":1427397594302,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594309,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9650,"Shuffle Write Time":279478,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4350,"Index":342,"Attempt":0,"Launch Time":1427397594311,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4346,"Index":338,"Attempt":0,"Launch Time":1427397594304,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594311,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9651,"Shuffle Write Time":267661,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4351,"Index":343,"Attempt":0,"Launch Time":1427397594321,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4345,"Index":337,"Attempt":0,"Launch Time":1427397594303,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594321,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":16,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9651,"Shuffle Write Time":264449,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4352,"Index":344,"Attempt":0,"Launch Time":1427397594323,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4348,"Index":340,"Attempt":0,"Launch Time":1427397594309,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594323,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9649,"Shuffle Write Time":615195,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4353,"Index":345,"Attempt":0,"Launch Time":1427397594324,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4349,"Index":341,"Attempt":0,"Launch Time":1427397594309,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594324,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":467378,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4354,"Index":346,"Attempt":0,"Launch Time":1427397594324,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4350,"Index":342,"Attempt":0,"Launch Time":1427397594311,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594325,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":264201,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4355,"Index":347,"Attempt":0,"Launch Time":1427397594328,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4351,"Index":343,"Attempt":0,"Launch Time":1427397594321,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594328,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":266498,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4356,"Index":348,"Attempt":0,"Launch Time":1427397594329,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4352,"Index":344,"Attempt":0,"Launch Time":1427397594323,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594329,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":274157,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4357,"Index":349,"Attempt":0,"Launch Time":1427397594330,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4354,"Index":346,"Attempt":0,"Launch Time":1427397594324,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594330,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":258544,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4358,"Index":350,"Attempt":0,"Launch Time":1427397594332,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4353,"Index":345,"Attempt":0,"Launch Time":1427397594324,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594332,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":284593,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4359,"Index":351,"Attempt":0,"Launch Time":1427397594336,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4355,"Index":347,"Attempt":0,"Launch Time":1427397594328,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594336,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":265571,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4360,"Index":352,"Attempt":0,"Launch Time":1427397594336,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4356,"Index":348,"Attempt":0,"Launch Time":1427397594329,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594336,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":271228,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4361,"Index":353,"Attempt":0,"Launch Time":1427397594337,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4357,"Index":349,"Attempt":0,"Launch Time":1427397594330,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594337,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":267539,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4362,"Index":354,"Attempt":0,"Launch Time":1427397594338,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4358,"Index":350,"Attempt":0,"Launch Time":1427397594332,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594338,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":280237,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4363,"Index":355,"Attempt":0,"Launch Time":1427397594343,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4364,"Index":356,"Attempt":0,"Launch Time":1427397594343,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4360,"Index":352,"Attempt":0,"Launch Time":1427397594336,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594343,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":288643,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4359,"Index":351,"Attempt":0,"Launch Time":1427397594336,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594343,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":370081,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4365,"Index":357,"Attempt":0,"Launch Time":1427397594344,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4361,"Index":353,"Attempt":0,"Launch Time":1427397594337,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594344,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":264746,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4366,"Index":358,"Attempt":0,"Launch Time":1427397594344,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4362,"Index":354,"Attempt":0,"Launch Time":1427397594338,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594344,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":271008,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4367,"Index":359,"Attempt":0,"Launch Time":1427397594350,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4363,"Index":355,"Attempt":0,"Launch Time":1427397594343,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594350,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":314053,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4368,"Index":360,"Attempt":0,"Launch Time":1427397594350,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4364,"Index":356,"Attempt":0,"Launch Time":1427397594343,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594350,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":324269,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4369,"Index":361,"Attempt":0,"Launch Time":1427397594352,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4322,"Index":314,"Attempt":0,"Launch Time":1427397594269,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594352,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":40,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":318902,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4370,"Index":362,"Attempt":0,"Launch Time":1427397594352,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4365,"Index":357,"Attempt":0,"Launch Time":1427397594344,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594352,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":377010,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4371,"Index":363,"Attempt":0,"Launch Time":1427397594354,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4366,"Index":358,"Attempt":0,"Launch Time":1427397594344,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594354,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":479847,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4372,"Index":364,"Attempt":0,"Launch Time":1427397594358,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4347,"Index":339,"Attempt":0,"Launch Time":1427397594307,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594358,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":50,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9647,"Shuffle Write Time":278845,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4373,"Index":365,"Attempt":0,"Launch Time":1427397594358,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4367,"Index":359,"Attempt":0,"Launch Time":1427397594350,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594358,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":302782,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4368,"Index":360,"Attempt":0,"Launch Time":1427397594350,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594358,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":346840,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4374,"Index":366,"Attempt":0,"Launch Time":1427397594358,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4375,"Index":367,"Attempt":0,"Launch Time":1427397594360,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4370,"Index":362,"Attempt":0,"Launch Time":1427397594352,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594360,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":285451,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4376,"Index":368,"Attempt":0,"Launch Time":1427397594361,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4371,"Index":363,"Attempt":0,"Launch Time":1427397594354,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594361,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":274073,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4377,"Index":369,"Attempt":0,"Launch Time":1427397594364,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4372,"Index":364,"Attempt":0,"Launch Time":1427397594358,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594367,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":270879,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4378,"Index":370,"Attempt":0,"Launch Time":1427397594367,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4379,"Index":371,"Attempt":0,"Launch Time":1427397594367,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4374,"Index":366,"Attempt":0,"Launch Time":1427397594358,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594367,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":280447,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4319,"Index":311,"Attempt":0,"Launch Time":1427397594261,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594367,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":103,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":562891,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4376,"Index":368,"Attempt":0,"Launch Time":1427397594361,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594367,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":256403,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4380,"Index":372,"Attempt":0,"Launch Time":1427397594367,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4381,"Index":373,"Attempt":0,"Launch Time":1427397594367,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4382,"Index":374,"Attempt":0,"Launch Time":1427397594368,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4373,"Index":365,"Attempt":0,"Launch Time":1427397594358,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594368,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":269276,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4369,"Index":361,"Attempt":0,"Launch Time":1427397594352,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594368,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":526869,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4383,"Index":375,"Attempt":0,"Launch Time":1427397594368,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4375,"Index":367,"Attempt":0,"Launch Time":1427397594360,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594368,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":283117,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4384,"Index":376,"Attempt":0,"Launch Time":1427397594372,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4378,"Index":370,"Attempt":0,"Launch Time":1427397594367,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594372,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":269364,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4385,"Index":377,"Attempt":0,"Launch Time":1427397594374,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4381,"Index":373,"Attempt":0,"Launch Time":1427397594367,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594374,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":267475,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4386,"Index":378,"Attempt":0,"Launch Time":1427397594374,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4383,"Index":375,"Attempt":0,"Launch Time":1427397594368,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594374,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":263811,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4387,"Index":379,"Attempt":0,"Launch Time":1427397594376,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4380,"Index":372,"Attempt":0,"Launch Time":1427397594367,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594376,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":376281,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4388,"Index":380,"Attempt":0,"Launch Time":1427397594378,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4384,"Index":376,"Attempt":0,"Launch Time":1427397594372,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594379,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":258413,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4389,"Index":381,"Attempt":0,"Launch Time":1427397594380,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4382,"Index":374,"Attempt":0,"Launch Time":1427397594368,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594380,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9612,"Shuffle Write Time":290355,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4390,"Index":382,"Attempt":0,"Launch Time":1427397594380,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4386,"Index":378,"Attempt":0,"Launch Time":1427397594374,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594380,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":296618,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4391,"Index":383,"Attempt":0,"Launch Time":1427397594381,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4385,"Index":377,"Attempt":0,"Launch Time":1427397594374,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594381,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":300719,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4392,"Index":384,"Attempt":0,"Launch Time":1427397594384,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4387,"Index":379,"Attempt":0,"Launch Time":1427397594376,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594384,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":260443,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4393,"Index":385,"Attempt":0,"Launch Time":1427397594385,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4388,"Index":380,"Attempt":0,"Launch Time":1427397594378,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594385,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":270453,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4394,"Index":386,"Attempt":0,"Launch Time":1427397594387,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4390,"Index":382,"Attempt":0,"Launch Time":1427397594380,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594387,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9614,"Shuffle Write Time":282791,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4391,"Index":383,"Attempt":0,"Launch Time":1427397594381,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594387,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":296949,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4395,"Index":387,"Attempt":0,"Launch Time":1427397594387,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4396,"Index":388,"Attempt":0,"Launch Time":1427397594387,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4389,"Index":381,"Attempt":0,"Launch Time":1427397594380,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594388,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":319738,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4397,"Index":389,"Attempt":0,"Launch Time":1427397594395,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4323,"Index":315,"Attempt":0,"Launch Time":1427397594269,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594395,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":38,"Executor Run Time":45,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":390014,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4398,"Index":390,"Attempt":0,"Launch Time":1427397594399,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4379,"Index":371,"Attempt":0,"Launch Time":1427397594367,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594399,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":284893,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4399,"Index":391,"Attempt":0,"Launch Time":1427397594402,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4377,"Index":369,"Attempt":0,"Launch Time":1427397594364,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594402,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":19,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1446908,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4400,"Index":392,"Attempt":0,"Launch Time":1427397594403,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4395,"Index":387,"Attempt":0,"Launch Time":1427397594387,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594403,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":6,"Executor Run Time":8,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":314006,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4401,"Index":393,"Attempt":0,"Launch Time":1427397594404,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4397,"Index":389,"Attempt":0,"Launch Time":1427397594395,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594405,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":350497,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4402,"Index":394,"Attempt":0,"Launch Time":1427397594405,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4396,"Index":388,"Attempt":0,"Launch Time":1427397594387,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594405,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":7,"Executor Run Time":10,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":960054,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4403,"Index":395,"Attempt":0,"Launch Time":1427397594407,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4398,"Index":390,"Attempt":0,"Launch Time":1427397594399,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594407,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9615,"Shuffle Write Time":297221,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4404,"Index":396,"Attempt":0,"Launch Time":1427397594408,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4399,"Index":391,"Attempt":0,"Launch Time":1427397594402,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594409,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":280209,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4405,"Index":397,"Attempt":0,"Launch Time":1427397594409,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4400,"Index":392,"Attempt":0,"Launch Time":1427397594403,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594409,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":249654,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4406,"Index":398,"Attempt":0,"Launch Time":1427397594410,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4401,"Index":393,"Attempt":0,"Launch Time":1427397594404,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594411,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":277643,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4407,"Index":399,"Attempt":0,"Launch Time":1427397594411,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4402,"Index":394,"Attempt":0,"Launch Time":1427397594405,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594411,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":245163,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4408,"Index":400,"Attempt":0,"Launch Time":1427397594414,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4403,"Index":395,"Attempt":0,"Launch Time":1427397594407,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594414,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":279418,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4409,"Index":401,"Attempt":0,"Launch Time":1427397594416,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4405,"Index":397,"Attempt":0,"Launch Time":1427397594409,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594416,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":274623,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4410,"Index":402,"Attempt":0,"Launch Time":1427397594417,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4411,"Index":403,"Attempt":0,"Launch Time":1427397594417,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4407,"Index":399,"Attempt":0,"Launch Time":1427397594411,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594417,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":269665,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4406,"Index":398,"Attempt":0,"Launch Time":1427397594410,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594417,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":299962,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4412,"Index":404,"Attempt":0,"Launch Time":1427397594422,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4409,"Index":401,"Attempt":0,"Launch Time":1427397594416,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594422,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":280342,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4413,"Index":405,"Attempt":0,"Launch Time":1427397594422,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4404,"Index":396,"Attempt":0,"Launch Time":1427397594408,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594422,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":280597,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4414,"Index":406,"Attempt":0,"Launch Time":1427397594423,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4411,"Index":403,"Attempt":0,"Launch Time":1427397594417,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594423,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":274170,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4415,"Index":407,"Attempt":0,"Launch Time":1427397594427,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4408,"Index":400,"Attempt":0,"Launch Time":1427397594414,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594427,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":286138,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4416,"Index":408,"Attempt":0,"Launch Time":1427397594429,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4414,"Index":406,"Attempt":0,"Launch Time":1427397594423,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594429,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":303096,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4417,"Index":409,"Attempt":0,"Launch Time":1427397594429,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4412,"Index":404,"Attempt":0,"Launch Time":1427397594422,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594429,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":310638,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4418,"Index":410,"Attempt":0,"Launch Time":1427397594431,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4413,"Index":405,"Attempt":0,"Launch Time":1427397594422,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594431,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":295894,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4419,"Index":411,"Attempt":0,"Launch Time":1427397594434,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4415,"Index":407,"Attempt":0,"Launch Time":1427397594427,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594434,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":355320,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4420,"Index":412,"Attempt":0,"Launch Time":1427397594435,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4421,"Index":413,"Attempt":0,"Launch Time":1427397594435,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4410,"Index":402,"Attempt":0,"Launch Time":1427397594417,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594435,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":369718,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4393,"Index":385,"Attempt":0,"Launch Time":1427397594385,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594435,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":49,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":671473,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4422,"Index":414,"Attempt":0,"Launch Time":1427397594438,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4418,"Index":410,"Attempt":0,"Launch Time":1427397594431,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594438,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":349078,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4423,"Index":415,"Attempt":0,"Launch Time":1427397594439,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4417,"Index":409,"Attempt":0,"Launch Time":1427397594429,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594440,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":435782,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4424,"Index":416,"Attempt":0,"Launch Time":1427397594442,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4420,"Index":412,"Attempt":0,"Launch Time":1427397594435,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594442,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":300098,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4425,"Index":417,"Attempt":0,"Launch Time":1427397594443,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4421,"Index":413,"Attempt":0,"Launch Time":1427397594435,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594443,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":313645,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4426,"Index":418,"Attempt":0,"Launch Time":1427397594443,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4416,"Index":408,"Attempt":0,"Launch Time":1427397594429,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594443,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":262755,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4427,"Index":419,"Attempt":0,"Launch Time":1427397594444,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4419,"Index":411,"Attempt":0,"Launch Time":1427397594434,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594445,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":338366,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4428,"Index":420,"Attempt":0,"Launch Time":1427397594445,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4423,"Index":415,"Attempt":0,"Launch Time":1427397594439,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594446,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":370241,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4429,"Index":421,"Attempt":0,"Launch Time":1427397594448,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4422,"Index":414,"Attempt":0,"Launch Time":1427397594438,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594448,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":258932,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4430,"Index":422,"Attempt":0,"Launch Time":1427397594449,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4425,"Index":417,"Attempt":0,"Launch Time":1427397594443,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594449,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":260708,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4431,"Index":423,"Attempt":0,"Launch Time":1427397594449,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4426,"Index":418,"Attempt":0,"Launch Time":1427397594443,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594449,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":256452,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4432,"Index":424,"Attempt":0,"Launch Time":1427397594451,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4428,"Index":420,"Attempt":0,"Launch Time":1427397594445,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594452,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":268913,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4433,"Index":425,"Attempt":0,"Launch Time":1427397594452,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4424,"Index":416,"Attempt":0,"Launch Time":1427397594442,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594452,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":276654,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4434,"Index":426,"Attempt":0,"Launch Time":1427397594453,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4427,"Index":419,"Attempt":0,"Launch Time":1427397594444,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594453,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":277793,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4435,"Index":427,"Attempt":0,"Launch Time":1427397594455,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4430,"Index":422,"Attempt":0,"Launch Time":1427397594449,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594455,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":303297,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4436,"Index":428,"Attempt":0,"Launch Time":1427397594456,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4431,"Index":423,"Attempt":0,"Launch Time":1427397594449,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594456,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":294053,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4437,"Index":429,"Attempt":0,"Launch Time":1427397594457,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4429,"Index":421,"Attempt":0,"Launch Time":1427397594448,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594457,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":261479,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4438,"Index":430,"Attempt":0,"Launch Time":1427397594458,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4439,"Index":431,"Attempt":0,"Launch Time":1427397594458,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4434,"Index":426,"Attempt":0,"Launch Time":1427397594453,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594459,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":265308,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4433,"Index":425,"Attempt":0,"Launch Time":1427397594452,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594459,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":271421,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4440,"Index":432,"Attempt":0,"Launch Time":1427397594459,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4432,"Index":424,"Attempt":0,"Launch Time":1427397594451,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594459,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":278634,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4441,"Index":433,"Attempt":0,"Launch Time":1427397594472,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4437,"Index":429,"Attempt":0,"Launch Time":1427397594457,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594472,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":338486,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4442,"Index":434,"Attempt":0,"Launch Time":1427397594472,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4439,"Index":431,"Attempt":0,"Launch Time":1427397594458,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594472,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":375314,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4443,"Index":435,"Attempt":0,"Launch Time":1427397594474,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4438,"Index":430,"Attempt":0,"Launch Time":1427397594458,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594474,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":14,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":282690,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4444,"Index":436,"Attempt":0,"Launch Time":1427397594478,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4436,"Index":428,"Attempt":0,"Launch Time":1427397594456,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594478,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":22,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":338130,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4445,"Index":437,"Attempt":0,"Launch Time":1427397594478,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4435,"Index":427,"Attempt":0,"Launch Time":1427397594455,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594478,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":23,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":414144,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4446,"Index":438,"Attempt":0,"Launch Time":1427397594480,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4441,"Index":433,"Attempt":0,"Launch Time":1427397594472,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594480,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":333383,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4447,"Index":439,"Attempt":0,"Launch Time":1427397594483,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4442,"Index":434,"Attempt":0,"Launch Time":1427397594472,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594484,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":674596,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4448,"Index":440,"Attempt":0,"Launch Time":1427397594484,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4443,"Index":435,"Attempt":0,"Launch Time":1427397594474,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594484,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":321046,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4449,"Index":441,"Attempt":0,"Launch Time":1427397594486,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4450,"Index":442,"Attempt":0,"Launch Time":1427397594486,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4444,"Index":436,"Attempt":0,"Launch Time":1427397594478,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594486,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":337944,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4445,"Index":437,"Attempt":0,"Launch Time":1427397594478,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594486,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":394149,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4451,"Index":443,"Attempt":0,"Launch Time":1427397594488,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4446,"Index":438,"Attempt":0,"Launch Time":1427397594480,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594489,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":300503,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4452,"Index":444,"Attempt":0,"Launch Time":1427397594492,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4450,"Index":442,"Attempt":0,"Launch Time":1427397594486,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594492,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":258203,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4453,"Index":445,"Attempt":0,"Launch Time":1427397594494,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4440,"Index":432,"Attempt":0,"Launch Time":1427397594459,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594494,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":34,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":572631,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4454,"Index":446,"Attempt":0,"Launch Time":1427397594496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4448,"Index":440,"Attempt":0,"Launch Time":1427397594484,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594496,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":328403,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4455,"Index":447,"Attempt":0,"Launch Time":1427397594496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4451,"Index":443,"Attempt":0,"Launch Time":1427397594488,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594496,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":280295,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4456,"Index":448,"Attempt":0,"Launch Time":1427397594497,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4449,"Index":441,"Attempt":0,"Launch Time":1427397594486,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594497,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1197457,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4457,"Index":449,"Attempt":0,"Launch Time":1427397594497,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4447,"Index":439,"Attempt":0,"Launch Time":1427397594483,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594497,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":481396,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4458,"Index":450,"Attempt":0,"Launch Time":1427397594498,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4452,"Index":444,"Attempt":0,"Launch Time":1427397594492,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594498,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":290887,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4459,"Index":451,"Attempt":0,"Launch Time":1427397594501,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4392,"Index":384,"Attempt":0,"Launch Time":1427397594384,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594501,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":116,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":316724,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4460,"Index":452,"Attempt":0,"Launch Time":1427397594504,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4456,"Index":448,"Attempt":0,"Launch Time":1427397594497,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594504,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":271343,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4461,"Index":453,"Attempt":0,"Launch Time":1427397594504,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4454,"Index":446,"Attempt":0,"Launch Time":1427397594496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594504,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":261040,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4462,"Index":454,"Attempt":0,"Launch Time":1427397594504,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4458,"Index":450,"Attempt":0,"Launch Time":1427397594498,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594504,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":279548,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4463,"Index":455,"Attempt":0,"Launch Time":1427397594506,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4457,"Index":449,"Attempt":0,"Launch Time":1427397594497,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594506,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":272788,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4464,"Index":456,"Attempt":0,"Launch Time":1427397594507,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4455,"Index":447,"Attempt":0,"Launch Time":1427397594496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594507,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":308241,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4465,"Index":457,"Attempt":0,"Launch Time":1427397594510,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4462,"Index":454,"Attempt":0,"Launch Time":1427397594504,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594510,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":296221,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4466,"Index":458,"Attempt":0,"Launch Time":1427397594511,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4460,"Index":452,"Attempt":0,"Launch Time":1427397594504,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594511,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":299798,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4467,"Index":459,"Attempt":0,"Launch Time":1427397594515,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4394,"Index":386,"Attempt":0,"Launch Time":1427397594387,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594515,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":87,"Executor Run Time":40,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":309244,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4468,"Index":460,"Attempt":0,"Launch Time":1427397594516,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4465,"Index":457,"Attempt":0,"Launch Time":1427397594510,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594516,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":288438,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4469,"Index":461,"Attempt":0,"Launch Time":1427397594522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4467,"Index":459,"Attempt":0,"Launch Time":1427397594515,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594522,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":304384,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4470,"Index":462,"Attempt":0,"Launch Time":1427397594522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4459,"Index":451,"Attempt":0,"Launch Time":1427397594501,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594522,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":7,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":415690,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4471,"Index":463,"Attempt":0,"Launch Time":1427397594522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4468,"Index":460,"Attempt":0,"Launch Time":1427397594516,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594523,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":307892,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4472,"Index":464,"Attempt":0,"Launch Time":1427397594525,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4453,"Index":445,"Attempt":0,"Launch Time":1427397594494,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594525,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":27,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9605,"Shuffle Write Time":319462,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4473,"Index":465,"Attempt":0,"Launch Time":1427397594530,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4474,"Index":466,"Attempt":0,"Launch Time":1427397594530,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4475,"Index":467,"Attempt":0,"Launch Time":1427397594530,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4470,"Index":462,"Attempt":0,"Launch Time":1427397594522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594530,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":329652,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4471,"Index":463,"Attempt":0,"Launch Time":1427397594522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594530,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9607,"Shuffle Write Time":328838,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4469,"Index":461,"Attempt":0,"Launch Time":1427397594522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594530,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":340359,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4476,"Index":468,"Attempt":0,"Launch Time":1427397594534,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4472,"Index":464,"Attempt":0,"Launch Time":1427397594525,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594535,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":316439,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4477,"Index":469,"Attempt":0,"Launch Time":1427397594537,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4474,"Index":466,"Attempt":0,"Launch Time":1427397594530,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594537,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":376893,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4478,"Index":470,"Attempt":0,"Launch Time":1427397594537,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4473,"Index":465,"Attempt":0,"Launch Time":1427397594530,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594538,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":343847,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4479,"Index":471,"Attempt":0,"Launch Time":1427397594544,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4480,"Index":472,"Attempt":0,"Launch Time":1427397594544,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4478,"Index":470,"Attempt":0,"Launch Time":1427397594537,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594544,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":277966,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4477,"Index":469,"Attempt":0,"Launch Time":1427397594537,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594544,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":299997,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4481,"Index":473,"Attempt":0,"Launch Time":1427397594544,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4464,"Index":456,"Attempt":0,"Launch Time":1427397594507,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594544,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":36,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1185560,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4482,"Index":474,"Attempt":0,"Launch Time":1427397594544,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4475,"Index":467,"Attempt":0,"Launch Time":1427397594530,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594545,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":6,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":449100,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4483,"Index":475,"Attempt":0,"Launch Time":1427397594547,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4476,"Index":468,"Attempt":0,"Launch Time":1427397594534,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594548,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":287184,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4484,"Index":476,"Attempt":0,"Launch Time":1427397594555,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4479,"Index":471,"Attempt":0,"Launch Time":1427397594544,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594555,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9613,"Shuffle Write Time":317631,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4485,"Index":477,"Attempt":0,"Launch Time":1427397594556,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4482,"Index":474,"Attempt":0,"Launch Time":1427397594544,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594556,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":301797,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4486,"Index":478,"Attempt":0,"Launch Time":1427397594557,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4481,"Index":473,"Attempt":0,"Launch Time":1427397594544,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594557,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":273365,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4487,"Index":479,"Attempt":0,"Launch Time":1427397594557,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4480,"Index":472,"Attempt":0,"Launch Time":1427397594544,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594557,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":7114742,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4488,"Index":480,"Attempt":0,"Launch Time":1427397594562,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4484,"Index":476,"Attempt":0,"Launch Time":1427397594555,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594562,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":276598,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4489,"Index":481,"Attempt":0,"Launch Time":1427397594563,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4483,"Index":475,"Attempt":0,"Launch Time":1427397594547,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594563,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":6,"Executor Run Time":8,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":237463,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4490,"Index":482,"Attempt":0,"Launch Time":1427397594563,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4491,"Index":483,"Attempt":0,"Launch Time":1427397594564,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4487,"Index":479,"Attempt":0,"Launch Time":1427397594557,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594564,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":318394,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4485,"Index":477,"Attempt":0,"Launch Time":1427397594556,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594564,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":453428,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4492,"Index":484,"Attempt":0,"Launch Time":1427397594568,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4488,"Index":480,"Attempt":0,"Launch Time":1427397594562,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594568,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":272582,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4493,"Index":485,"Attempt":0,"Launch Time":1427397594569,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4494,"Index":486,"Attempt":0,"Launch Time":1427397594570,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4491,"Index":483,"Attempt":0,"Launch Time":1427397594564,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594570,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":242881,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4490,"Index":482,"Attempt":0,"Launch Time":1427397594563,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594570,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":367542,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4495,"Index":487,"Attempt":0,"Launch Time":1427397594570,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4489,"Index":481,"Attempt":0,"Launch Time":1427397594563,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594570,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":283454,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4496,"Index":488,"Attempt":0,"Launch Time":1427397594570,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4463,"Index":455,"Attempt":0,"Launch Time":1427397594506,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594570,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":62,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":290822,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4497,"Index":489,"Attempt":0,"Launch Time":1427397594573,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4486,"Index":478,"Attempt":0,"Launch Time":1427397594557,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594573,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":271865,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4498,"Index":490,"Attempt":0,"Launch Time":1427397594574,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4492,"Index":484,"Attempt":0,"Launch Time":1427397594568,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594574,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":281435,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4499,"Index":491,"Attempt":0,"Launch Time":1427397594575,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4500,"Index":492,"Attempt":0,"Launch Time":1427397594576,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4493,"Index":485,"Attempt":0,"Launch Time":1427397594569,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594576,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":246401,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4495,"Index":487,"Attempt":0,"Launch Time":1427397594570,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594576,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":244495,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4501,"Index":493,"Attempt":0,"Launch Time":1427397594576,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4496,"Index":488,"Attempt":0,"Launch Time":1427397594570,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594577,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":280975,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4502,"Index":494,"Attempt":0,"Launch Time":1427397594578,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4494,"Index":486,"Attempt":0,"Launch Time":1427397594570,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594578,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":295045,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4503,"Index":495,"Attempt":0,"Launch Time":1427397594582,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4498,"Index":490,"Attempt":0,"Launch Time":1427397594574,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594583,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":332631,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4504,"Index":496,"Attempt":0,"Launch Time":1427397594583,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4500,"Index":492,"Attempt":0,"Launch Time":1427397594576,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594584,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":310518,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4505,"Index":497,"Attempt":0,"Launch Time":1427397594585,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4499,"Index":491,"Attempt":0,"Launch Time":1427397594575,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594585,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":475796,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4506,"Index":498,"Attempt":0,"Launch Time":1427397594585,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4502,"Index":494,"Attempt":0,"Launch Time":1427397594578,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594585,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":313747,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4507,"Index":499,"Attempt":0,"Launch Time":1427397594589,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4497,"Index":489,"Attempt":0,"Launch Time":1427397594573,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594589,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":311988,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4508,"Index":500,"Attempt":0,"Launch Time":1427397594590,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4509,"Index":501,"Attempt":0,"Launch Time":1427397594590,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4503,"Index":495,"Attempt":0,"Launch Time":1427397594582,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594590,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":312912,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4504,"Index":496,"Attempt":0,"Launch Time":1427397594583,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594590,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":273561,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4510,"Index":502,"Attempt":0,"Launch Time":1427397594591,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4501,"Index":493,"Attempt":0,"Launch Time":1427397594576,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594591,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":2261520,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4511,"Index":503,"Attempt":0,"Launch Time":1427397594592,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4506,"Index":498,"Attempt":0,"Launch Time":1427397594585,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594592,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":277692,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4512,"Index":504,"Attempt":0,"Launch Time":1427397594592,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4505,"Index":497,"Attempt":0,"Launch Time":1427397594585,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594592,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":268462,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4513,"Index":505,"Attempt":0,"Launch Time":1427397594596,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4509,"Index":501,"Attempt":0,"Launch Time":1427397594590,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594596,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":262471,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4514,"Index":506,"Attempt":0,"Launch Time":1427397594596,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4507,"Index":499,"Attempt":0,"Launch Time":1427397594589,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594596,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":250443,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4515,"Index":507,"Attempt":0,"Launch Time":1427397594598,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4511,"Index":503,"Attempt":0,"Launch Time":1427397594592,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594598,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":275318,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4516,"Index":508,"Attempt":0,"Launch Time":1427397594599,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4512,"Index":504,"Attempt":0,"Launch Time":1427397594592,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594599,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":271864,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4517,"Index":509,"Attempt":0,"Launch Time":1427397594600,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4510,"Index":502,"Attempt":0,"Launch Time":1427397594591,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594600,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":280716,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4518,"Index":510,"Attempt":0,"Launch Time":1427397594601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4508,"Index":500,"Attempt":0,"Launch Time":1427397594590,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594601,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":607026,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4519,"Index":511,"Attempt":0,"Launch Time":1427397594602,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4513,"Index":505,"Attempt":0,"Launch Time":1427397594596,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594602,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":292029,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4520,"Index":512,"Attempt":0,"Launch Time":1427397594604,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4515,"Index":507,"Attempt":0,"Launch Time":1427397594598,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594605,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":284359,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4521,"Index":513,"Attempt":0,"Launch Time":1427397594606,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4517,"Index":509,"Attempt":0,"Launch Time":1427397594600,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594606,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":284069,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4522,"Index":514,"Attempt":0,"Launch Time":1427397594607,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4516,"Index":508,"Attempt":0,"Launch Time":1427397594599,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594607,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":290435,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4523,"Index":515,"Attempt":0,"Launch Time":1427397594608,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4519,"Index":511,"Attempt":0,"Launch Time":1427397594602,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594609,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":259250,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4524,"Index":516,"Attempt":0,"Launch Time":1427397594612,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4525,"Index":517,"Attempt":0,"Launch Time":1427397594612,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4521,"Index":513,"Attempt":0,"Launch Time":1427397594606,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594613,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":271315,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4520,"Index":512,"Attempt":0,"Launch Time":1427397594604,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594613,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":270891,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4526,"Index":518,"Attempt":0,"Launch Time":1427397594614,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4523,"Index":515,"Attempt":0,"Launch Time":1427397594608,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594615,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":304711,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4527,"Index":519,"Attempt":0,"Launch Time":1427397594618,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4518,"Index":510,"Attempt":0,"Launch Time":1427397594601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594618,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":275504,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4528,"Index":520,"Attempt":0,"Launch Time":1427397594618,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4522,"Index":514,"Attempt":0,"Launch Time":1427397594607,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594618,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":5035572,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4529,"Index":521,"Attempt":0,"Launch Time":1427397594619,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4525,"Index":517,"Attempt":0,"Launch Time":1427397594612,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594619,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":289288,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4530,"Index":522,"Attempt":0,"Launch Time":1427397594620,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4524,"Index":516,"Attempt":0,"Launch Time":1427397594612,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594620,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":281465,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4531,"Index":523,"Attempt":0,"Launch Time":1427397594625,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4532,"Index":524,"Attempt":0,"Launch Time":1427397594625,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4526,"Index":518,"Attempt":0,"Launch Time":1427397594614,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594625,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":288610,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4514,"Index":506,"Attempt":0,"Launch Time":1427397594596,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594626,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":27,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":300471,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4533,"Index":525,"Attempt":0,"Launch Time":1427397594626,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4461,"Index":453,"Attempt":0,"Launch Time":1427397594504,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594627,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":121,"Result Size":930,"JVM GC Time":8,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":340571,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4534,"Index":526,"Attempt":0,"Launch Time":1427397594631,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4535,"Index":527,"Attempt":0,"Launch Time":1427397594632,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4536,"Index":528,"Attempt":0,"Launch Time":1427397594632,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4527,"Index":519,"Attempt":0,"Launch Time":1427397594618,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594632,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":277894,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4528,"Index":520,"Attempt":0,"Launch Time":1427397594618,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594633,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":303478,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4531,"Index":523,"Attempt":0,"Launch Time":1427397594625,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594634,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":311274,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4537,"Index":529,"Attempt":0,"Launch Time":1427397594640,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4534,"Index":526,"Attempt":0,"Launch Time":1427397594631,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594640,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":291794,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4538,"Index":530,"Attempt":0,"Launch Time":1427397594641,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4539,"Index":531,"Attempt":0,"Launch Time":1427397594641,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4540,"Index":532,"Attempt":0,"Launch Time":1427397594641,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4535,"Index":527,"Attempt":0,"Launch Time":1427397594632,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594641,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":305723,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4536,"Index":528,"Attempt":0,"Launch Time":1427397594632,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594641,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":266744,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4466,"Index":458,"Attempt":0,"Launch Time":1427397594511,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594641,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":127,"Result Size":930,"JVM GC Time":8,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":595143,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4541,"Index":533,"Attempt":0,"Launch Time":1427397594644,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4530,"Index":522,"Attempt":0,"Launch Time":1427397594620,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594644,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":16,"Executor Run Time":7,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":293972,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4542,"Index":534,"Attempt":0,"Launch Time":1427397594645,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4533,"Index":525,"Attempt":0,"Launch Time":1427397594626,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594645,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":297777,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4543,"Index":535,"Attempt":0,"Launch Time":1427397594648,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4532,"Index":524,"Attempt":0,"Launch Time":1427397594625,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594648,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":5,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":335031,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4544,"Index":536,"Attempt":0,"Launch Time":1427397594651,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4541,"Index":533,"Attempt":0,"Launch Time":1427397594644,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594651,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":310970,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4545,"Index":537,"Attempt":0,"Launch Time":1427397594652,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4537,"Index":529,"Attempt":0,"Launch Time":1427397594640,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594652,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":349792,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4546,"Index":538,"Attempt":0,"Launch Time":1427397594652,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4542,"Index":534,"Attempt":0,"Launch Time":1427397594645,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594652,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":339492,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4547,"Index":539,"Attempt":0,"Launch Time":1427397594664,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4539,"Index":531,"Attempt":0,"Launch Time":1427397594641,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594664,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":288838,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4548,"Index":540,"Attempt":0,"Launch Time":1427397594664,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4543,"Index":535,"Attempt":0,"Launch Time":1427397594648,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594664,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":279553,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4549,"Index":541,"Attempt":0,"Launch Time":1427397594669,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4540,"Index":532,"Attempt":0,"Launch Time":1427397594641,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594669,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":288927,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4550,"Index":542,"Attempt":0,"Launch Time":1427397594670,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4548,"Index":540,"Attempt":0,"Launch Time":1427397594664,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594670,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":291213,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4551,"Index":543,"Attempt":0,"Launch Time":1427397594675,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4545,"Index":537,"Attempt":0,"Launch Time":1427397594652,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594675,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":23,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":304236,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4552,"Index":544,"Attempt":0,"Launch Time":1427397594677,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4544,"Index":536,"Attempt":0,"Launch Time":1427397594651,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594677,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":26,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":295832,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4553,"Index":545,"Attempt":0,"Launch Time":1427397594677,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4550,"Index":542,"Attempt":0,"Launch Time":1427397594670,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594677,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":299758,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4554,"Index":546,"Attempt":0,"Launch Time":1427397594681,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4546,"Index":538,"Attempt":0,"Launch Time":1427397594652,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594681,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":27,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":841769,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4555,"Index":547,"Attempt":0,"Launch Time":1427397594682,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4551,"Index":543,"Attempt":0,"Launch Time":1427397594675,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594682,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":244258,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4556,"Index":548,"Attempt":0,"Launch Time":1427397594683,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4547,"Index":539,"Attempt":0,"Launch Time":1427397594664,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594683,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":260161,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4557,"Index":549,"Attempt":0,"Launch Time":1427397594684,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4553,"Index":545,"Attempt":0,"Launch Time":1427397594677,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594684,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":265224,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4558,"Index":550,"Attempt":0,"Launch Time":1427397594684,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4552,"Index":544,"Attempt":0,"Launch Time":1427397594677,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594685,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":276358,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4559,"Index":551,"Attempt":0,"Launch Time":1427397594688,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4554,"Index":546,"Attempt":0,"Launch Time":1427397594681,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594688,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":290295,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4560,"Index":552,"Attempt":0,"Launch Time":1427397594689,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4529,"Index":521,"Attempt":0,"Launch Time":1427397594619,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594690,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":17,"Executor Run Time":36,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":303203,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4561,"Index":553,"Attempt":0,"Launch Time":1427397594691,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4557,"Index":549,"Attempt":0,"Launch Time":1427397594684,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594691,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":304521,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4562,"Index":554,"Attempt":0,"Launch Time":1427397594691,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4556,"Index":548,"Attempt":0,"Launch Time":1427397594683,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594691,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":277293,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4563,"Index":555,"Attempt":0,"Launch Time":1427397594692,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4555,"Index":547,"Attempt":0,"Launch Time":1427397594682,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594692,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":289961,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4564,"Index":556,"Attempt":0,"Launch Time":1427397594693,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4558,"Index":550,"Attempt":0,"Launch Time":1427397594684,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594693,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":303606,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4565,"Index":557,"Attempt":0,"Launch Time":1427397594697,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4538,"Index":530,"Attempt":0,"Launch Time":1427397594641,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594698,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":32,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":306740,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4566,"Index":558,"Attempt":0,"Launch Time":1427397594698,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4549,"Index":541,"Attempt":0,"Launch Time":1427397594669,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594698,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":9,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":307002,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4567,"Index":559,"Attempt":0,"Launch Time":1427397594698,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4568,"Index":560,"Attempt":0,"Launch Time":1427397594699,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4561,"Index":553,"Attempt":0,"Launch Time":1427397594691,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594699,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":335482,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4559,"Index":551,"Attempt":0,"Launch Time":1427397594688,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594700,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":323623,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4569,"Index":561,"Attempt":0,"Launch Time":1427397594703,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4564,"Index":556,"Attempt":0,"Launch Time":1427397594693,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594703,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":265132,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4570,"Index":562,"Attempt":0,"Launch Time":1427397594704,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4566,"Index":558,"Attempt":0,"Launch Time":1427397594698,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594704,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":228605,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4571,"Index":563,"Attempt":0,"Launch Time":1427397594704,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4568,"Index":560,"Attempt":0,"Launch Time":1427397594699,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594704,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":233577,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4572,"Index":564,"Attempt":0,"Launch Time":1427397594707,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4567,"Index":559,"Attempt":0,"Launch Time":1427397594698,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594708,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":295950,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4573,"Index":565,"Attempt":0,"Launch Time":1427397594709,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4562,"Index":554,"Attempt":0,"Launch Time":1427397594691,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594709,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":257751,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4574,"Index":566,"Attempt":0,"Launch Time":1427397594710,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4571,"Index":563,"Attempt":0,"Launch Time":1427397594704,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594711,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":265407,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4575,"Index":567,"Attempt":0,"Launch Time":1427397594711,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4570,"Index":562,"Attempt":0,"Launch Time":1427397594704,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594711,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":274763,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4576,"Index":568,"Attempt":0,"Launch Time":1427397594711,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4563,"Index":555,"Attempt":0,"Launch Time":1427397594692,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594711,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":278164,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4577,"Index":569,"Attempt":0,"Launch Time":1427397594718,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4572,"Index":564,"Attempt":0,"Launch Time":1427397594707,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594718,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":296899,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4578,"Index":570,"Attempt":0,"Launch Time":1427397594722,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4565,"Index":557,"Attempt":0,"Launch Time":1427397594697,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594722,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":313910,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4579,"Index":571,"Attempt":0,"Launch Time":1427397594722,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4573,"Index":565,"Attempt":0,"Launch Time":1427397594709,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594722,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":426213,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4580,"Index":572,"Attempt":0,"Launch Time":1427397594723,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4576,"Index":568,"Attempt":0,"Launch Time":1427397594711,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594723,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":5,"Executor Run Time":7,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":361481,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4581,"Index":573,"Attempt":0,"Launch Time":1427397594726,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4577,"Index":569,"Attempt":0,"Launch Time":1427397594718,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594726,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":267243,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4582,"Index":574,"Attempt":0,"Launch Time":1427397594726,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4583,"Index":575,"Attempt":0,"Launch Time":1427397594727,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4575,"Index":567,"Attempt":0,"Launch Time":1427397594711,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594727,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":5,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":241318,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4574,"Index":566,"Attempt":0,"Launch Time":1427397594710,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594727,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":264682,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4584,"Index":576,"Attempt":0,"Launch Time":1427397594730,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4578,"Index":570,"Attempt":0,"Launch Time":1427397594722,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594730,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":286722,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4585,"Index":577,"Attempt":0,"Launch Time":1427397594730,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4580,"Index":572,"Attempt":0,"Launch Time":1427397594723,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594730,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9614,"Shuffle Write Time":331989,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4586,"Index":578,"Attempt":0,"Launch Time":1427397594733,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4569,"Index":561,"Attempt":0,"Launch Time":1427397594703,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594733,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":28,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":275846,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4587,"Index":579,"Attempt":0,"Launch Time":1427397594733,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4581,"Index":573,"Attempt":0,"Launch Time":1427397594726,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594733,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":278184,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4588,"Index":580,"Attempt":0,"Launch Time":1427397594733,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4560,"Index":552,"Attempt":0,"Launch Time":1427397594689,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594734,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":41,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":6535811,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4589,"Index":581,"Attempt":0,"Launch Time":1427397594736,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4585,"Index":577,"Attempt":0,"Launch Time":1427397594730,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594736,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":294135,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4590,"Index":582,"Attempt":0,"Launch Time":1427397594737,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4584,"Index":576,"Attempt":0,"Launch Time":1427397594730,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594737,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":295089,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4591,"Index":583,"Attempt":0,"Launch Time":1427397594740,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4582,"Index":574,"Attempt":0,"Launch Time":1427397594726,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594740,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":770061,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4592,"Index":584,"Attempt":0,"Launch Time":1427397594740,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4587,"Index":579,"Attempt":0,"Launch Time":1427397594733,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594740,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":339235,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4593,"Index":585,"Attempt":0,"Launch Time":1427397594741,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4583,"Index":575,"Attempt":0,"Launch Time":1427397594727,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594741,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":375322,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4594,"Index":586,"Attempt":0,"Launch Time":1427397594741,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4588,"Index":580,"Attempt":0,"Launch Time":1427397594733,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594741,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9611,"Shuffle Write Time":310864,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4595,"Index":587,"Attempt":0,"Launch Time":1427397594742,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4589,"Index":581,"Attempt":0,"Launch Time":1427397594736,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594742,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":284200,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4596,"Index":588,"Attempt":0,"Launch Time":1427397594744,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4590,"Index":582,"Attempt":0,"Launch Time":1427397594737,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594744,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":289178,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4597,"Index":589,"Attempt":0,"Launch Time":1427397594749,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4593,"Index":585,"Attempt":0,"Launch Time":1427397594741,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594749,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":272612,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4598,"Index":590,"Attempt":0,"Launch Time":1427397594749,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4599,"Index":591,"Attempt":0,"Launch Time":1427397594749,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4595,"Index":587,"Attempt":0,"Launch Time":1427397594742,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594750,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":441864,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4592,"Index":584,"Attempt":0,"Launch Time":1427397594740,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594750,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":269676,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4600,"Index":592,"Attempt":0,"Launch Time":1427397594751,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4591,"Index":583,"Attempt":0,"Launch Time":1427397594740,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594751,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":305175,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4601,"Index":593,"Attempt":0,"Launch Time":1427397594754,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4596,"Index":588,"Attempt":0,"Launch Time":1427397594744,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594754,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9608,"Shuffle Write Time":270958,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4602,"Index":594,"Attempt":0,"Launch Time":1427397594755,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4594,"Index":586,"Attempt":0,"Launch Time":1427397594741,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594755,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":235924,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4603,"Index":595,"Attempt":0,"Launch Time":1427397594756,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4598,"Index":590,"Attempt":0,"Launch Time":1427397594749,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594756,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":279801,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4604,"Index":596,"Attempt":0,"Launch Time":1427397594757,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4599,"Index":591,"Attempt":0,"Launch Time":1427397594749,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594757,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":407310,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4605,"Index":597,"Attempt":0,"Launch Time":1427397594760,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4601,"Index":593,"Attempt":0,"Launch Time":1427397594754,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594760,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":280546,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4606,"Index":598,"Attempt":0,"Launch Time":1427397594760,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4597,"Index":589,"Attempt":0,"Launch Time":1427397594749,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594761,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":282408,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4607,"Index":599,"Attempt":0,"Launch Time":1427397594763,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4604,"Index":596,"Attempt":0,"Launch Time":1427397594757,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594763,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":267505,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4608,"Index":600,"Attempt":0,"Launch Time":1427397594764,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4586,"Index":578,"Attempt":0,"Launch Time":1427397594733,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594764,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":26,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":310125,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4609,"Index":601,"Attempt":0,"Launch Time":1427397594767,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4605,"Index":597,"Attempt":0,"Launch Time":1427397594760,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594767,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":300322,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4610,"Index":602,"Attempt":0,"Launch Time":1427397594768,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4579,"Index":571,"Attempt":0,"Launch Time":1427397594722,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594768,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":26,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":302400,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4611,"Index":603,"Attempt":0,"Launch Time":1427397594770,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4602,"Index":594,"Attempt":0,"Launch Time":1427397594755,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594771,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":295043,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4612,"Index":604,"Attempt":0,"Launch Time":1427397594771,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4608,"Index":600,"Attempt":0,"Launch Time":1427397594764,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594771,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":285864,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4613,"Index":605,"Attempt":0,"Launch Time":1427397594774,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4610,"Index":602,"Attempt":0,"Launch Time":1427397594768,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594774,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":267994,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4614,"Index":606,"Attempt":0,"Launch Time":1427397594777,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4615,"Index":607,"Attempt":0,"Launch Time":1427397594778,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4612,"Index":604,"Attempt":0,"Launch Time":1427397594771,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594778,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":307837,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4611,"Index":603,"Attempt":0,"Launch Time":1427397594770,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594778,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":289997,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4616,"Index":608,"Attempt":0,"Launch Time":1427397594779,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4603,"Index":595,"Attempt":0,"Launch Time":1427397594756,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594779,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":342585,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4617,"Index":609,"Attempt":0,"Launch Time":1427397594781,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4613,"Index":605,"Attempt":0,"Launch Time":1427397594774,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594781,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":322401,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4618,"Index":610,"Attempt":0,"Launch Time":1427397594785,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4615,"Index":607,"Attempt":0,"Launch Time":1427397594778,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594785,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":329836,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4619,"Index":611,"Attempt":0,"Launch Time":1427397594787,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4614,"Index":606,"Attempt":0,"Launch Time":1427397594777,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594787,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":334072,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4620,"Index":612,"Attempt":0,"Launch Time":1427397594788,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4616,"Index":608,"Attempt":0,"Launch Time":1427397594779,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594788,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":301921,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4621,"Index":613,"Attempt":0,"Launch Time":1427397594789,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4617,"Index":609,"Attempt":0,"Launch Time":1427397594781,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594790,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":389606,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4622,"Index":614,"Attempt":0,"Launch Time":1427397594797,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4618,"Index":610,"Attempt":0,"Launch Time":1427397594785,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594797,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":3,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":292132,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4623,"Index":615,"Attempt":0,"Launch Time":1427397594800,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4619,"Index":611,"Attempt":0,"Launch Time":1427397594787,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594800,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":3,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":343916,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4624,"Index":616,"Attempt":0,"Launch Time":1427397594801,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4621,"Index":613,"Attempt":0,"Launch Time":1427397594789,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594801,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":5,"Executor Run Time":6,"Result Size":930,"JVM GC Time":3,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":299795,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4625,"Index":617,"Attempt":0,"Launch Time":1427397594804,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4620,"Index":612,"Attempt":0,"Launch Time":1427397594788,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594804,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":3,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":272277,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4626,"Index":618,"Attempt":0,"Launch Time":1427397594806,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4622,"Index":614,"Attempt":0,"Launch Time":1427397594797,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594806,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":346752,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4627,"Index":619,"Attempt":0,"Launch Time":1427397594807,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4623,"Index":615,"Attempt":0,"Launch Time":1427397594800,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594807,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":281762,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4628,"Index":620,"Attempt":0,"Launch Time":1427397594808,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4624,"Index":616,"Attempt":0,"Launch Time":1427397594801,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594808,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":276282,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4629,"Index":621,"Attempt":0,"Launch Time":1427397594810,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4625,"Index":617,"Attempt":0,"Launch Time":1427397594804,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594810,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":277247,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4630,"Index":622,"Attempt":0,"Launch Time":1427397594814,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4626,"Index":618,"Attempt":0,"Launch Time":1427397594806,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594814,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":304255,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4631,"Index":623,"Attempt":0,"Launch Time":1427397594815,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4627,"Index":619,"Attempt":0,"Launch Time":1427397594807,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594815,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":286669,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4632,"Index":624,"Attempt":0,"Launch Time":1427397594815,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4628,"Index":620,"Attempt":0,"Launch Time":1427397594808,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594815,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":293250,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4633,"Index":625,"Attempt":0,"Launch Time":1427397594816,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4629,"Index":621,"Attempt":0,"Launch Time":1427397594810,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594816,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":306135,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4634,"Index":626,"Attempt":0,"Launch Time":1427397594818,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4609,"Index":601,"Attempt":0,"Launch Time":1427397594767,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594819,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":6,"Executor Run Time":28,"Result Size":930,"JVM GC Time":3,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":300190,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4635,"Index":627,"Attempt":0,"Launch Time":1427397594819,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4600,"Index":592,"Attempt":0,"Launch Time":1427397594751,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594819,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":25,"Executor Run Time":39,"Result Size":930,"JVM GC Time":3,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":3423963,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4636,"Index":628,"Attempt":0,"Launch Time":1427397594822,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4630,"Index":622,"Attempt":0,"Launch Time":1427397594814,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594822,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":370562,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4637,"Index":629,"Attempt":0,"Launch Time":1427397594824,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4631,"Index":623,"Attempt":0,"Launch Time":1427397594815,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594824,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":364008,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4638,"Index":630,"Attempt":0,"Launch Time":1427397594825,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4632,"Index":624,"Attempt":0,"Launch Time":1427397594815,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594825,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":325347,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4639,"Index":631,"Attempt":0,"Launch Time":1427397594826,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4634,"Index":626,"Attempt":0,"Launch Time":1427397594818,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594826,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":316028,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4640,"Index":632,"Attempt":0,"Launch Time":1427397594827,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4633,"Index":625,"Attempt":0,"Launch Time":1427397594816,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594827,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":800522,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4641,"Index":633,"Attempt":0,"Launch Time":1427397594829,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4636,"Index":628,"Attempt":0,"Launch Time":1427397594822,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594829,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":286977,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4642,"Index":634,"Attempt":0,"Launch Time":1427397594831,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4638,"Index":630,"Attempt":0,"Launch Time":1427397594825,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594831,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":279934,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4643,"Index":635,"Attempt":0,"Launch Time":1427397594832,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4637,"Index":629,"Attempt":0,"Launch Time":1427397594824,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594832,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":276100,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4644,"Index":636,"Attempt":0,"Launch Time":1427397594833,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4639,"Index":631,"Attempt":0,"Launch Time":1427397594826,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594833,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":285940,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4641,"Index":633,"Attempt":0,"Launch Time":1427397594829,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594838,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":293603,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4645,"Index":637,"Attempt":0,"Launch Time":1427397594838,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4646,"Index":638,"Attempt":0,"Launch Time":1427397594839,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4644,"Index":636,"Attempt":0,"Launch Time":1427397594833,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594839,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":310581,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4647,"Index":639,"Attempt":0,"Launch Time":1427397594840,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4643,"Index":635,"Attempt":0,"Launch Time":1427397594832,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594840,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":298614,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4648,"Index":640,"Attempt":0,"Launch Time":1427397594841,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4642,"Index":634,"Attempt":0,"Launch Time":1427397594831,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594841,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":309498,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4649,"Index":641,"Attempt":0,"Launch Time":1427397594846,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4646,"Index":638,"Attempt":0,"Launch Time":1427397594839,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594846,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":305945,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4650,"Index":642,"Attempt":0,"Launch Time":1427397594847,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4645,"Index":637,"Attempt":0,"Launch Time":1427397594838,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594847,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":399734,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4651,"Index":643,"Attempt":0,"Launch Time":1427397594847,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4647,"Index":639,"Attempt":0,"Launch Time":1427397594840,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594847,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":296878,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4652,"Index":644,"Attempt":0,"Launch Time":1427397594847,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4648,"Index":640,"Attempt":0,"Launch Time":1427397594841,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594848,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":281583,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4653,"Index":645,"Attempt":0,"Launch Time":1427397594853,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4649,"Index":641,"Attempt":0,"Launch Time":1427397594846,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594853,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":310199,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4654,"Index":646,"Attempt":0,"Launch Time":1427397594855,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4651,"Index":643,"Attempt":0,"Launch Time":1427397594847,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594855,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":281493,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4650,"Index":642,"Attempt":0,"Launch Time":1427397594847,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594856,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":293609,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4655,"Index":647,"Attempt":0,"Launch Time":1427397594856,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4656,"Index":648,"Attempt":0,"Launch Time":1427397594859,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4640,"Index":632,"Attempt":0,"Launch Time":1427397594827,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594859,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":31,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":410372,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4657,"Index":649,"Attempt":0,"Launch Time":1427397594861,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4653,"Index":645,"Attempt":0,"Launch Time":1427397594853,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594861,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":320466,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4658,"Index":650,"Attempt":0,"Launch Time":1427397594861,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4659,"Index":651,"Attempt":0,"Launch Time":1427397594861,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4654,"Index":646,"Attempt":0,"Launch Time":1427397594855,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594861,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":268623,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4652,"Index":644,"Attempt":0,"Launch Time":1427397594847,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594861,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":288399,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4660,"Index":652,"Attempt":0,"Launch Time":1427397594864,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4655,"Index":647,"Attempt":0,"Launch Time":1427397594856,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594864,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":317966,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4661,"Index":653,"Attempt":0,"Launch Time":1427397594868,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4659,"Index":651,"Attempt":0,"Launch Time":1427397594861,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594868,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":341361,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4662,"Index":654,"Attempt":0,"Launch Time":1427397594869,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4657,"Index":649,"Attempt":0,"Launch Time":1427397594861,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594869,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":335988,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4663,"Index":655,"Attempt":0,"Launch Time":1427397594869,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4658,"Index":650,"Attempt":0,"Launch Time":1427397594861,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594869,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":362102,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4660,"Index":652,"Attempt":0,"Launch Time":1427397594864,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594872,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":304089,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4664,"Index":656,"Attempt":0,"Launch Time":1427397594873,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4665,"Index":657,"Attempt":0,"Launch Time":1427397594880,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4656,"Index":648,"Attempt":0,"Launch Time":1427397594859,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594880,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":345163,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4666,"Index":658,"Attempt":0,"Launch Time":1427397594881,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4662,"Index":654,"Attempt":0,"Launch Time":1427397594869,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594881,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":310436,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4667,"Index":659,"Attempt":0,"Launch Time":1427397594883,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4663,"Index":655,"Attempt":0,"Launch Time":1427397594869,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594883,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":295388,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4668,"Index":660,"Attempt":0,"Launch Time":1427397594884,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4664,"Index":656,"Attempt":0,"Launch Time":1427397594873,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594885,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":272387,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4669,"Index":661,"Attempt":0,"Launch Time":1427397594887,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4666,"Index":658,"Attempt":0,"Launch Time":1427397594881,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594887,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":272143,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4670,"Index":662,"Attempt":0,"Launch Time":1427397594894,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4607,"Index":599,"Attempt":0,"Launch Time":1427397594763,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594894,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":5,"Executor Run Time":99,"Result Size":930,"JVM GC Time":8,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":282605,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4671,"Index":663,"Attempt":0,"Launch Time":1427397594896,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4606,"Index":598,"Attempt":0,"Launch Time":1427397594760,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594896,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":102,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":292395,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4672,"Index":664,"Attempt":0,"Launch Time":1427397594901,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4670,"Index":662,"Attempt":0,"Launch Time":1427397594894,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594901,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":415632,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4673,"Index":665,"Attempt":0,"Launch Time":1427397594901,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4635,"Index":627,"Attempt":0,"Launch Time":1427397594819,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594902,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":67,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":379788,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4674,"Index":666,"Attempt":0,"Launch Time":1427397594907,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4672,"Index":664,"Attempt":0,"Launch Time":1427397594901,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594907,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":320996,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4675,"Index":667,"Attempt":0,"Launch Time":1427397594908,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4676,"Index":668,"Attempt":0,"Launch Time":1427397594908,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4673,"Index":665,"Attempt":0,"Launch Time":1427397594901,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594908,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":298044,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4671,"Index":663,"Attempt":0,"Launch Time":1427397594896,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594908,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":298146,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4677,"Index":669,"Attempt":0,"Launch Time":1427397594914,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4667,"Index":659,"Attempt":0,"Launch Time":1427397594883,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594914,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":29,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":302395,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4678,"Index":670,"Attempt":0,"Launch Time":1427397594914,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4679,"Index":671,"Attempt":0,"Launch Time":1427397594914,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4676,"Index":668,"Attempt":0,"Launch Time":1427397594908,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594914,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":289049,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4674,"Index":666,"Attempt":0,"Launch Time":1427397594907,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594914,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":301927,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4680,"Index":672,"Attempt":0,"Launch Time":1427397594915,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4675,"Index":667,"Attempt":0,"Launch Time":1427397594908,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594915,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":302874,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4681,"Index":673,"Attempt":0,"Launch Time":1427397594921,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4682,"Index":674,"Attempt":0,"Launch Time":1427397594921,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4679,"Index":671,"Attempt":0,"Launch Time":1427397594914,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594922,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":342425,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4678,"Index":670,"Attempt":0,"Launch Time":1427397594914,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594922,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":326799,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4683,"Index":675,"Attempt":0,"Launch Time":1427397594922,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4680,"Index":672,"Attempt":0,"Launch Time":1427397594915,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594922,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":365689,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4684,"Index":676,"Attempt":0,"Launch Time":1427397594923,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4677,"Index":669,"Attempt":0,"Launch Time":1427397594914,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594923,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":390836,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4685,"Index":677,"Attempt":0,"Launch Time":1427397594925,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4668,"Index":660,"Attempt":0,"Launch Time":1427397594884,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594925,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":39,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":271074,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4686,"Index":678,"Attempt":0,"Launch Time":1427397594925,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4669,"Index":661,"Attempt":0,"Launch Time":1427397594887,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594925,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":38,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":309194,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4687,"Index":679,"Attempt":0,"Launch Time":1427397594928,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4682,"Index":674,"Attempt":0,"Launch Time":1427397594921,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594928,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":292271,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4688,"Index":680,"Attempt":0,"Launch Time":1427397594928,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4681,"Index":673,"Attempt":0,"Launch Time":1427397594921,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594928,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":295179,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4689,"Index":681,"Attempt":0,"Launch Time":1427397594930,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4684,"Index":676,"Attempt":0,"Launch Time":1427397594923,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594930,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":288177,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4690,"Index":682,"Attempt":0,"Launch Time":1427397594932,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4683,"Index":675,"Attempt":0,"Launch Time":1427397594922,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594932,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":505448,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4691,"Index":683,"Attempt":0,"Launch Time":1427397594932,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4665,"Index":657,"Attempt":0,"Launch Time":1427397594880,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594933,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":280946,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4692,"Index":684,"Attempt":0,"Launch Time":1427397594933,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4687,"Index":679,"Attempt":0,"Launch Time":1427397594928,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594934,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":253877,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4693,"Index":685,"Attempt":0,"Launch Time":1427397594934,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4688,"Index":680,"Attempt":0,"Launch Time":1427397594928,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594934,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":276890,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4694,"Index":686,"Attempt":0,"Launch Time":1427397594936,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4689,"Index":681,"Attempt":0,"Launch Time":1427397594930,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594936,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":271754,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4695,"Index":687,"Attempt":0,"Launch Time":1427397594939,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4690,"Index":682,"Attempt":0,"Launch Time":1427397594932,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594939,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":285616,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4696,"Index":688,"Attempt":0,"Launch Time":1427397594939,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4691,"Index":683,"Attempt":0,"Launch Time":1427397594932,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594939,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":291053,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4697,"Index":689,"Attempt":0,"Launch Time":1427397594943,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4692,"Index":684,"Attempt":0,"Launch Time":1427397594933,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594943,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":337316,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4698,"Index":690,"Attempt":0,"Launch Time":1427397594943,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4694,"Index":686,"Attempt":0,"Launch Time":1427397594936,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594943,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":336430,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4699,"Index":691,"Attempt":0,"Launch Time":1427397594944,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4693,"Index":685,"Attempt":0,"Launch Time":1427397594934,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594945,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":381769,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4700,"Index":692,"Attempt":0,"Launch Time":1427397594946,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4696,"Index":688,"Attempt":0,"Launch Time":1427397594939,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594946,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":301371,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4701,"Index":693,"Attempt":0,"Launch Time":1427397594949,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4697,"Index":689,"Attempt":0,"Launch Time":1427397594943,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594949,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":292961,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4702,"Index":694,"Attempt":0,"Launch Time":1427397594950,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4698,"Index":690,"Attempt":0,"Launch Time":1427397594943,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594950,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":260322,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4703,"Index":695,"Attempt":0,"Launch Time":1427397594953,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4700,"Index":692,"Attempt":0,"Launch Time":1427397594946,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594953,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":281395,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4704,"Index":696,"Attempt":0,"Launch Time":1427397594954,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4661,"Index":653,"Attempt":0,"Launch Time":1427397594868,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594954,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":86,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":391620,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4705,"Index":697,"Attempt":0,"Launch Time":1427397594955,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4706,"Index":698,"Attempt":0,"Launch Time":1427397594955,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4701,"Index":693,"Attempt":0,"Launch Time":1427397594949,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594956,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":284584,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4699,"Index":691,"Attempt":0,"Launch Time":1427397594944,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594956,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":347340,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4707,"Index":699,"Attempt":0,"Launch Time":1427397594956,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4702,"Index":694,"Attempt":0,"Launch Time":1427397594950,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594956,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":295337,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4708,"Index":700,"Attempt":0,"Launch Time":1427397594958,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4695,"Index":687,"Attempt":0,"Launch Time":1427397594939,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594958,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":19,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1018066,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4709,"Index":701,"Attempt":0,"Launch Time":1427397594959,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4703,"Index":695,"Attempt":0,"Launch Time":1427397594953,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594959,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":274940,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4710,"Index":702,"Attempt":0,"Launch Time":1427397594968,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4705,"Index":697,"Attempt":0,"Launch Time":1427397594955,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594968,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9612,"Shuffle Write Time":297831,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4711,"Index":703,"Attempt":0,"Launch Time":1427397594968,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4686,"Index":678,"Attempt":0,"Launch Time":1427397594925,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594968,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":43,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":9089528,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4712,"Index":704,"Attempt":0,"Launch Time":1427397594973,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4713,"Index":705,"Attempt":0,"Launch Time":1427397594973,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4706,"Index":698,"Attempt":0,"Launch Time":1427397594955,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594973,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":10414784,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4707,"Index":699,"Attempt":0,"Launch Time":1427397594956,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594973,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":10426024,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4714,"Index":706,"Attempt":0,"Launch Time":1427397594974,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4710,"Index":702,"Attempt":0,"Launch Time":1427397594968,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594974,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":328629,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4715,"Index":707,"Attempt":0,"Launch Time":1427397594975,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4708,"Index":700,"Attempt":0,"Launch Time":1427397594958,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594975,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":301068,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4716,"Index":708,"Attempt":0,"Launch Time":1427397594976,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4709,"Index":701,"Attempt":0,"Launch Time":1427397594959,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594976,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":268140,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4717,"Index":709,"Attempt":0,"Launch Time":1427397594977,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4711,"Index":703,"Attempt":0,"Launch Time":1427397594968,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594977,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":291868,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4718,"Index":710,"Attempt":0,"Launch Time":1427397594980,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4719,"Index":711,"Attempt":0,"Launch Time":1427397594980,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4712,"Index":704,"Attempt":0,"Launch Time":1427397594973,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594980,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":287686,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4713,"Index":705,"Attempt":0,"Launch Time":1427397594973,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594980,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9615,"Shuffle Write Time":291390,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4720,"Index":712,"Attempt":0,"Launch Time":1427397594981,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4714,"Index":706,"Attempt":0,"Launch Time":1427397594974,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594981,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":258690,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4721,"Index":713,"Attempt":0,"Launch Time":1427397594984,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4716,"Index":708,"Attempt":0,"Launch Time":1427397594976,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594984,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":335105,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4722,"Index":714,"Attempt":0,"Launch Time":1427397594986,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4718,"Index":710,"Attempt":0,"Launch Time":1427397594980,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594986,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":282020,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4723,"Index":715,"Attempt":0,"Launch Time":1427397594987,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4720,"Index":712,"Attempt":0,"Launch Time":1427397594981,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594987,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":251203,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4724,"Index":716,"Attempt":0,"Launch Time":1427397594988,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4719,"Index":711,"Attempt":0,"Launch Time":1427397594980,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594988,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":292869,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4725,"Index":717,"Attempt":0,"Launch Time":1427397594990,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4721,"Index":713,"Attempt":0,"Launch Time":1427397594984,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594991,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9613,"Shuffle Write Time":299388,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4726,"Index":718,"Attempt":0,"Launch Time":1427397594994,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4722,"Index":714,"Attempt":0,"Launch Time":1427397594986,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594994,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1293355,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4727,"Index":719,"Attempt":0,"Launch Time":1427397595005,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4724,"Index":716,"Attempt":0,"Launch Time":1427397594988,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595006,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":335275,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4728,"Index":720,"Attempt":0,"Launch Time":1427397595017,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4729,"Index":721,"Attempt":0,"Launch Time":1427397595018,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4730,"Index":722,"Attempt":0,"Launch Time":1427397595018,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4731,"Index":723,"Attempt":0,"Launch Time":1427397595018,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4704,"Index":696,"Attempt":0,"Launch Time":1427397594954,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595018,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":36,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":306556,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4723,"Index":715,"Attempt":0,"Launch Time":1427397594987,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595018,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":351228,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4732,"Index":724,"Attempt":0,"Launch Time":1427397595018,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4727,"Index":719,"Attempt":0,"Launch Time":1427397595005,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595018,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":302933,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4685,"Index":677,"Attempt":0,"Launch Time":1427397594925,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595018,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":39,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":582616,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4726,"Index":718,"Attempt":0,"Launch Time":1427397594994,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595019,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":326472,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4725,"Index":717,"Attempt":0,"Launch Time":1427397594990,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595019,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":348577,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4733,"Index":725,"Attempt":0,"Launch Time":1427397595019,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4734,"Index":726,"Attempt":0,"Launch Time":1427397595024,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4735,"Index":727,"Attempt":0,"Launch Time":1427397595024,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4715,"Index":707,"Attempt":0,"Launch Time":1427397594975,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595024,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":14,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":274948,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4717,"Index":709,"Attempt":0,"Launch Time":1427397594977,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595024,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":330099,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4736,"Index":728,"Attempt":0,"Launch Time":1427397595027,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4730,"Index":722,"Attempt":0,"Launch Time":1427397595018,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595027,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":279983,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4737,"Index":729,"Attempt":0,"Launch Time":1427397595031,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4734,"Index":726,"Attempt":0,"Launch Time":1427397595024,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595031,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":271176,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4738,"Index":730,"Attempt":0,"Launch Time":1427397595034,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4736,"Index":728,"Attempt":0,"Launch Time":1427397595027,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595034,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":317389,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4739,"Index":731,"Attempt":0,"Launch Time":1427397595035,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4732,"Index":724,"Attempt":0,"Launch Time":1427397595018,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595035,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":428120,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4740,"Index":732,"Attempt":0,"Launch Time":1427397595040,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4737,"Index":729,"Attempt":0,"Launch Time":1427397595031,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595040,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":320574,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4741,"Index":733,"Attempt":0,"Launch Time":1427397595041,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4738,"Index":730,"Attempt":0,"Launch Time":1427397595034,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595042,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":318414,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4742,"Index":734,"Attempt":0,"Launch Time":1427397595043,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4735,"Index":727,"Attempt":0,"Launch Time":1427397595024,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595044,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":282370,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4743,"Index":735,"Attempt":0,"Launch Time":1427397595045,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4740,"Index":732,"Attempt":0,"Launch Time":1427397595040,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595045,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":276662,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4744,"Index":736,"Attempt":0,"Launch Time":1427397595046,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4739,"Index":731,"Attempt":0,"Launch Time":1427397595035,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595046,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":307321,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4745,"Index":737,"Attempt":0,"Launch Time":1427397595052,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4741,"Index":733,"Attempt":0,"Launch Time":1427397595041,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595052,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":281311,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4746,"Index":738,"Attempt":0,"Launch Time":1427397595052,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4744,"Index":736,"Attempt":0,"Launch Time":1427397595046,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595053,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":314913,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4747,"Index":739,"Attempt":0,"Launch Time":1427397595054,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4743,"Index":735,"Attempt":0,"Launch Time":1427397595045,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595054,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":295222,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4748,"Index":740,"Attempt":0,"Launch Time":1427397595055,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4742,"Index":734,"Attempt":0,"Launch Time":1427397595043,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595056,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":273524,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4749,"Index":741,"Attempt":0,"Launch Time":1427397595058,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4745,"Index":737,"Attempt":0,"Launch Time":1427397595052,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595058,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":263187,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4750,"Index":742,"Attempt":0,"Launch Time":1427397595060,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4746,"Index":738,"Attempt":0,"Launch Time":1427397595052,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595060,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":493093,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4751,"Index":743,"Attempt":0,"Launch Time":1427397595062,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4747,"Index":739,"Attempt":0,"Launch Time":1427397595054,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595063,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":265015,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4752,"Index":744,"Attempt":0,"Launch Time":1427397595064,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4749,"Index":741,"Attempt":0,"Launch Time":1427397595058,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595064,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":251216,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4753,"Index":745,"Attempt":0,"Launch Time":1427397595067,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4750,"Index":742,"Attempt":0,"Launch Time":1427397595060,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595067,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":304380,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4754,"Index":746,"Attempt":0,"Launch Time":1427397595069,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4748,"Index":740,"Attempt":0,"Launch Time":1427397595055,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595069,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1538385,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4755,"Index":747,"Attempt":0,"Launch Time":1427397595072,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4751,"Index":743,"Attempt":0,"Launch Time":1427397595062,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595072,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":388688,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4756,"Index":748,"Attempt":0,"Launch Time":1427397595073,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4752,"Index":744,"Attempt":0,"Launch Time":1427397595064,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595073,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":338472,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4757,"Index":749,"Attempt":0,"Launch Time":1427397595074,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4753,"Index":745,"Attempt":0,"Launch Time":1427397595067,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595085,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":379690,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4758,"Index":750,"Attempt":0,"Launch Time":1427397595085,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4754,"Index":746,"Attempt":0,"Launch Time":1427397595069,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595086,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":310332,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4759,"Index":751,"Attempt":0,"Launch Time":1427397595087,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4731,"Index":723,"Attempt":0,"Launch Time":1427397595018,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595087,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":18,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":337114,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4760,"Index":752,"Attempt":0,"Launch Time":1427397595091,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4761,"Index":753,"Attempt":0,"Launch Time":1427397595091,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4733,"Index":725,"Attempt":0,"Launch Time":1427397595019,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595091,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":289673,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4755,"Index":747,"Attempt":0,"Launch Time":1427397595072,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595091,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":263126,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4762,"Index":754,"Attempt":0,"Launch Time":1427397595091,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4758,"Index":750,"Attempt":0,"Launch Time":1427397595085,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595092,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":266885,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4763,"Index":755,"Attempt":0,"Launch Time":1427397595093,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4756,"Index":748,"Attempt":0,"Launch Time":1427397595073,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595093,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":7,"Executor Run Time":13,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1006719,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4764,"Index":756,"Attempt":0,"Launch Time":1427397595094,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4757,"Index":749,"Attempt":0,"Launch Time":1427397595074,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595094,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":289715,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4765,"Index":757,"Attempt":0,"Launch Time":1427397595094,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4728,"Index":720,"Attempt":0,"Launch Time":1427397595017,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595094,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":4033611,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4766,"Index":758,"Attempt":0,"Launch Time":1427397595095,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4759,"Index":751,"Attempt":0,"Launch Time":1427397595087,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595095,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":285611,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4767,"Index":759,"Attempt":0,"Launch Time":1427397595097,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4762,"Index":754,"Attempt":0,"Launch Time":1427397595091,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595098,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":273241,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4768,"Index":760,"Attempt":0,"Launch Time":1427397595098,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4760,"Index":752,"Attempt":0,"Launch Time":1427397595091,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595098,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":278797,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4769,"Index":761,"Attempt":0,"Launch Time":1427397595100,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4770,"Index":762,"Attempt":0,"Launch Time":1427397595101,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4763,"Index":755,"Attempt":0,"Launch Time":1427397595093,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595101,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":291495,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4765,"Index":757,"Attempt":0,"Launch Time":1427397595094,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595101,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":295193,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4771,"Index":763,"Attempt":0,"Launch Time":1427397595101,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4729,"Index":721,"Attempt":0,"Launch Time":1427397595018,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595101,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":14,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":490152,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4772,"Index":764,"Attempt":0,"Launch Time":1427397595104,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4768,"Index":760,"Attempt":0,"Launch Time":1427397595098,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595104,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":272521,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4773,"Index":765,"Attempt":0,"Launch Time":1427397595104,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4767,"Index":759,"Attempt":0,"Launch Time":1427397595097,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595104,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":308214,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4774,"Index":766,"Attempt":0,"Launch Time":1427397595107,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4775,"Index":767,"Attempt":0,"Launch Time":1427397595107,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4771,"Index":763,"Attempt":0,"Launch Time":1427397595101,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595107,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":278757,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4769,"Index":761,"Attempt":0,"Launch Time":1427397595100,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595107,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":274476,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4776,"Index":768,"Attempt":0,"Launch Time":1427397595108,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4770,"Index":762,"Attempt":0,"Launch Time":1427397595101,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595108,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":240805,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4777,"Index":769,"Attempt":0,"Launch Time":1427397595109,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4772,"Index":764,"Attempt":0,"Launch Time":1427397595104,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595109,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":270265,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4778,"Index":770,"Attempt":0,"Launch Time":1427397595113,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4774,"Index":766,"Attempt":0,"Launch Time":1427397595107,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595114,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":273188,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4779,"Index":771,"Attempt":0,"Launch Time":1427397595114,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4775,"Index":767,"Attempt":0,"Launch Time":1427397595107,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595114,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":286239,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4780,"Index":772,"Attempt":0,"Launch Time":1427397595115,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4776,"Index":768,"Attempt":0,"Launch Time":1427397595108,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595115,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":273824,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4781,"Index":773,"Attempt":0,"Launch Time":1427397595115,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4777,"Index":769,"Attempt":0,"Launch Time":1427397595109,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595115,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":240871,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4782,"Index":774,"Attempt":0,"Launch Time":1427397595120,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4778,"Index":770,"Attempt":0,"Launch Time":1427397595113,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595120,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":334314,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4783,"Index":775,"Attempt":0,"Launch Time":1427397595120,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4779,"Index":771,"Attempt":0,"Launch Time":1427397595114,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595120,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":324488,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4784,"Index":776,"Attempt":0,"Launch Time":1427397595121,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4781,"Index":773,"Attempt":0,"Launch Time":1427397595115,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595121,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":321540,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4785,"Index":777,"Attempt":0,"Launch Time":1427397595125,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4773,"Index":765,"Attempt":0,"Launch Time":1427397595104,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595125,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":299744,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4786,"Index":778,"Attempt":0,"Launch Time":1427397595126,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4780,"Index":772,"Attempt":0,"Launch Time":1427397595115,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595126,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":317123,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4787,"Index":779,"Attempt":0,"Launch Time":1427397595128,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4788,"Index":780,"Attempt":0,"Launch Time":1427397595128,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4783,"Index":775,"Attempt":0,"Launch Time":1427397595120,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595128,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":324591,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4766,"Index":758,"Attempt":0,"Launch Time":1427397595095,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595128,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":33,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":412446,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4789,"Index":781,"Attempt":0,"Launch Time":1427397595129,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4784,"Index":776,"Attempt":0,"Launch Time":1427397595121,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595129,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":308969,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4790,"Index":782,"Attempt":0,"Launch Time":1427397595132,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4785,"Index":777,"Attempt":0,"Launch Time":1427397595125,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595132,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":313009,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4791,"Index":783,"Attempt":0,"Launch Time":1427397595134,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4786,"Index":778,"Attempt":0,"Launch Time":1427397595126,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595134,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":324965,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4792,"Index":784,"Attempt":0,"Launch Time":1427397595135,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4782,"Index":774,"Attempt":0,"Launch Time":1427397595120,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595135,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":2954636,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4793,"Index":785,"Attempt":0,"Launch Time":1427397595143,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4788,"Index":780,"Attempt":0,"Launch Time":1427397595128,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595143,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":382005,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4794,"Index":786,"Attempt":0,"Launch Time":1427397595143,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4789,"Index":781,"Attempt":0,"Launch Time":1427397595129,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595144,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":380933,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4795,"Index":787,"Attempt":0,"Launch Time":1427397595146,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4790,"Index":782,"Attempt":0,"Launch Time":1427397595132,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595146,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":337810,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4761,"Index":753,"Attempt":0,"Launch Time":1427397595091,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595146,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":50,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":299843,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4796,"Index":788,"Attempt":0,"Launch Time":1427397595146,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4797,"Index":789,"Attempt":0,"Launch Time":1427397595146,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4798,"Index":790,"Attempt":0,"Launch Time":1427397595146,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4787,"Index":779,"Attempt":0,"Launch Time":1427397595128,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595146,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":300202,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4764,"Index":756,"Attempt":0,"Launch Time":1427397595094,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595146,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":284986,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4799,"Index":791,"Attempt":0,"Launch Time":1427397595147,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4800,"Index":792,"Attempt":0,"Launch Time":1427397595147,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4792,"Index":784,"Attempt":0,"Launch Time":1427397595135,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595147,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":237939,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4791,"Index":783,"Attempt":0,"Launch Time":1427397595134,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595147,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":319083,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4801,"Index":793,"Attempt":0,"Launch Time":1427397595163,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4798,"Index":790,"Attempt":0,"Launch Time":1427397595146,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595163,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":285497,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4802,"Index":794,"Attempt":0,"Launch Time":1427397595163,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4803,"Index":795,"Attempt":0,"Launch Time":1427397595163,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4799,"Index":791,"Attempt":0,"Launch Time":1427397595147,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595163,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":330535,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4797,"Index":789,"Attempt":0,"Launch Time":1427397595146,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595163,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":15,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":311758,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4804,"Index":796,"Attempt":0,"Launch Time":1427397595164,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4800,"Index":792,"Attempt":0,"Launch Time":1427397595147,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595164,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":305238,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4805,"Index":797,"Attempt":0,"Launch Time":1427397595174,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4803,"Index":795,"Attempt":0,"Launch Time":1427397595163,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595174,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":415798,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4806,"Index":798,"Attempt":0,"Launch Time":1427397595174,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4802,"Index":794,"Attempt":0,"Launch Time":1427397595163,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595175,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":276793,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4807,"Index":799,"Attempt":0,"Launch Time":1427397595176,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4801,"Index":793,"Attempt":0,"Launch Time":1427397595163,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595176,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":283862,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4808,"Index":800,"Attempt":0,"Launch Time":1427397595177,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4804,"Index":796,"Attempt":0,"Launch Time":1427397595164,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595177,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":298639,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4809,"Index":801,"Attempt":0,"Launch Time":1427397595178,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4796,"Index":788,"Attempt":0,"Launch Time":1427397595146,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595178,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":289907,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4810,"Index":802,"Attempt":0,"Launch Time":1427397595180,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4805,"Index":797,"Attempt":0,"Launch Time":1427397595174,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595181,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":302370,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4811,"Index":803,"Attempt":0,"Launch Time":1427397595184,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4808,"Index":800,"Attempt":0,"Launch Time":1427397595177,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595185,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":308557,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4812,"Index":804,"Attempt":0,"Launch Time":1427397595187,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4806,"Index":798,"Attempt":0,"Launch Time":1427397595174,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595187,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":353705,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4813,"Index":805,"Attempt":0,"Launch Time":1427397595188,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4810,"Index":802,"Attempt":0,"Launch Time":1427397595180,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595188,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":316760,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4814,"Index":806,"Attempt":0,"Launch Time":1427397595189,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4807,"Index":799,"Attempt":0,"Launch Time":1427397595176,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595189,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":344310,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4815,"Index":807,"Attempt":0,"Launch Time":1427397595191,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4811,"Index":803,"Attempt":0,"Launch Time":1427397595184,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595191,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":307338,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4816,"Index":808,"Attempt":0,"Launch Time":1427397595193,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4809,"Index":801,"Attempt":0,"Launch Time":1427397595178,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595193,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":259878,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4817,"Index":809,"Attempt":0,"Launch Time":1427397595194,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4818,"Index":810,"Attempt":0,"Launch Time":1427397595194,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4812,"Index":804,"Attempt":0,"Launch Time":1427397595187,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595194,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":288158,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4813,"Index":805,"Attempt":0,"Launch Time":1427397595188,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595194,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":264583,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4819,"Index":811,"Attempt":0,"Launch Time":1427397595195,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4814,"Index":806,"Attempt":0,"Launch Time":1427397595189,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595195,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":289669,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4820,"Index":812,"Attempt":0,"Launch Time":1427397595196,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4815,"Index":807,"Attempt":0,"Launch Time":1427397595191,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595196,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":278782,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4821,"Index":813,"Attempt":0,"Launch Time":1427397595200,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4818,"Index":810,"Attempt":0,"Launch Time":1427397595194,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595200,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":271556,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4822,"Index":814,"Attempt":0,"Launch Time":1427397595201,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4823,"Index":815,"Attempt":0,"Launch Time":1427397595203,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4824,"Index":816,"Attempt":0,"Launch Time":1427397595204,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4819,"Index":811,"Attempt":0,"Launch Time":1427397595195,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595205,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":303596,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4820,"Index":812,"Attempt":0,"Launch Time":1427397595196,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595205,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":587311,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4817,"Index":809,"Attempt":0,"Launch Time":1427397595194,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595206,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":407980,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4825,"Index":817,"Attempt":0,"Launch Time":1427397595207,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4821,"Index":813,"Attempt":0,"Launch Time":1427397595200,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595207,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":317554,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4826,"Index":818,"Attempt":0,"Launch Time":1427397595211,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4822,"Index":814,"Attempt":0,"Launch Time":1427397595201,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595211,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":306731,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4827,"Index":819,"Attempt":0,"Launch Time":1427397595211,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4828,"Index":820,"Attempt":0,"Launch Time":1427397595211,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4824,"Index":816,"Attempt":0,"Launch Time":1427397595204,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595211,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":296470,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4823,"Index":815,"Attempt":0,"Launch Time":1427397595203,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595212,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":376171,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4829,"Index":821,"Attempt":0,"Launch Time":1427397595217,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4830,"Index":822,"Attempt":0,"Launch Time":1427397595217,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4831,"Index":823,"Attempt":0,"Launch Time":1427397595217,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4827,"Index":819,"Attempt":0,"Launch Time":1427397595211,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595217,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":280070,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4828,"Index":820,"Attempt":0,"Launch Time":1427397595211,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595218,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":291060,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4826,"Index":818,"Attempt":0,"Launch Time":1427397595211,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595218,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":247770,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4832,"Index":824,"Attempt":0,"Launch Time":1427397595224,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4831,"Index":823,"Attempt":0,"Launch Time":1427397595217,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595224,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":284595,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4833,"Index":825,"Attempt":0,"Launch Time":1427397595224,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4830,"Index":822,"Attempt":0,"Launch Time":1427397595217,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595224,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":320532,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4834,"Index":826,"Attempt":0,"Launch Time":1427397595228,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4825,"Index":817,"Attempt":0,"Launch Time":1427397595207,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595228,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":11285036,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4835,"Index":827,"Attempt":0,"Launch Time":1427397595230,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4832,"Index":824,"Attempt":0,"Launch Time":1427397595224,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595230,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":293942,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4836,"Index":828,"Attempt":0,"Launch Time":1427397595231,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4833,"Index":825,"Attempt":0,"Launch Time":1427397595224,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595231,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":282758,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4837,"Index":829,"Attempt":0,"Launch Time":1427397595239,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4835,"Index":827,"Attempt":0,"Launch Time":1427397595230,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595239,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":391326,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4838,"Index":830,"Attempt":0,"Launch Time":1427397595240,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4834,"Index":826,"Attempt":0,"Launch Time":1427397595228,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595240,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":397240,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4839,"Index":831,"Attempt":0,"Launch Time":1427397595245,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4836,"Index":828,"Attempt":0,"Launch Time":1427397595231,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595246,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":2485150,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4840,"Index":832,"Attempt":0,"Launch Time":1427397595250,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4838,"Index":830,"Attempt":0,"Launch Time":1427397595240,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595250,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":350861,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4841,"Index":833,"Attempt":0,"Launch Time":1427397595252,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4837,"Index":829,"Attempt":0,"Launch Time":1427397595239,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595253,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":303257,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4842,"Index":834,"Attempt":0,"Launch Time":1427397595257,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4843,"Index":835,"Attempt":0,"Launch Time":1427397595257,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4840,"Index":832,"Attempt":0,"Launch Time":1427397595250,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595257,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":344128,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4839,"Index":831,"Attempt":0,"Launch Time":1427397595245,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595257,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":721284,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4844,"Index":836,"Attempt":0,"Launch Time":1427397595263,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4842,"Index":834,"Attempt":0,"Launch Time":1427397595257,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595263,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":294223,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4845,"Index":837,"Attempt":0,"Launch Time":1427397595268,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4795,"Index":787,"Attempt":0,"Launch Time":1427397595146,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595268,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":104,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1057437,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4846,"Index":838,"Attempt":0,"Launch Time":1427397595274,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4794,"Index":786,"Attempt":0,"Launch Time":1427397595143,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595274,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":113,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":298757,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4847,"Index":839,"Attempt":0,"Launch Time":1427397595275,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4829,"Index":821,"Attempt":0,"Launch Time":1427397595217,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595275,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":33,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":273545,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4848,"Index":840,"Attempt":0,"Launch Time":1427397595276,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4793,"Index":785,"Attempt":0,"Launch Time":1427397595143,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595292,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":113,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1121142,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4849,"Index":841,"Attempt":0,"Launch Time":1427397595292,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4846,"Index":838,"Attempt":0,"Launch Time":1427397595274,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595293,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":464691,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4850,"Index":842,"Attempt":0,"Launch Time":1427397595293,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4816,"Index":808,"Attempt":0,"Launch Time":1427397595193,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595309,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":25,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":309504,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4851,"Index":843,"Attempt":0,"Launch Time":1427397595310,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4852,"Index":844,"Attempt":0,"Launch Time":1427397595310,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4845,"Index":837,"Attempt":0,"Launch Time":1427397595268,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595310,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":24,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":501477,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4853,"Index":845,"Attempt":0,"Launch Time":1427397595310,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4854,"Index":846,"Attempt":0,"Launch Time":1427397595310,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4855,"Index":847,"Attempt":0,"Launch Time":1427397595311,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4844,"Index":836,"Attempt":0,"Launch Time":1427397595263,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595311,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":17,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":345970,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4849,"Index":841,"Attempt":0,"Launch Time":1427397595292,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595311,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":405145,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4841,"Index":833,"Attempt":0,"Launch Time":1427397595252,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595311,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":51,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":413061,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4848,"Index":840,"Attempt":0,"Launch Time":1427397595276,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595311,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":900368,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4856,"Index":848,"Attempt":0,"Launch Time":1427397595318,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4843,"Index":835,"Attempt":0,"Launch Time":1427397595257,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595318,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":60,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1903116,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4857,"Index":849,"Attempt":0,"Launch Time":1427397595319,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4854,"Index":846,"Attempt":0,"Launch Time":1427397595310,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595319,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":297974,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4858,"Index":850,"Attempt":0,"Launch Time":1427397595319,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4852,"Index":844,"Attempt":0,"Launch Time":1427397595310,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595319,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":300702,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4859,"Index":851,"Attempt":0,"Launch Time":1427397595322,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4853,"Index":845,"Attempt":0,"Launch Time":1427397595310,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595322,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":289449,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4860,"Index":852,"Attempt":0,"Launch Time":1427397595326,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4858,"Index":850,"Attempt":0,"Launch Time":1427397595319,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595326,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":338724,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4857,"Index":849,"Attempt":0,"Launch Time":1427397595319,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595330,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":512553,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4861,"Index":853,"Attempt":0,"Launch Time":1427397595330,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4862,"Index":854,"Attempt":0,"Launch Time":1427397595332,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4855,"Index":847,"Attempt":0,"Launch Time":1427397595311,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595332,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":299003,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4863,"Index":855,"Attempt":0,"Launch Time":1427397595340,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4860,"Index":852,"Attempt":0,"Launch Time":1427397595326,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595341,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":272303,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4864,"Index":856,"Attempt":0,"Launch Time":1427397595341,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4865,"Index":857,"Attempt":0,"Launch Time":1427397595341,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4866,"Index":858,"Attempt":0,"Launch Time":1427397595341,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4867,"Index":859,"Attempt":0,"Launch Time":1427397595341,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4861,"Index":853,"Attempt":0,"Launch Time":1427397595330,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595341,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":442445,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4859,"Index":851,"Attempt":0,"Launch Time":1427397595322,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595341,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":546805,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4868,"Index":860,"Attempt":0,"Launch Time":1427397595341,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4862,"Index":854,"Attempt":0,"Launch Time":1427397595332,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595341,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":300619,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4847,"Index":839,"Attempt":0,"Launch Time":1427397595275,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595341,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":26,"Executor Run Time":28,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":388881,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4850,"Index":842,"Attempt":0,"Launch Time":1427397595293,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595342,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":6,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":386805,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4869,"Index":861,"Attempt":0,"Launch Time":1427397595347,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4868,"Index":860,"Attempt":0,"Launch Time":1427397595341,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595347,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":244655,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4870,"Index":862,"Attempt":0,"Launch Time":1427397595347,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4865,"Index":857,"Attempt":0,"Launch Time":1427397595341,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595348,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":590745,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4871,"Index":863,"Attempt":0,"Launch Time":1427397595348,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4863,"Index":855,"Attempt":0,"Launch Time":1427397595340,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595348,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":283124,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4872,"Index":864,"Attempt":0,"Launch Time":1427397595350,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4867,"Index":859,"Attempt":0,"Launch Time":1427397595341,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595350,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":615228,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4873,"Index":865,"Attempt":0,"Launch Time":1427397595354,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4874,"Index":866,"Attempt":0,"Launch Time":1427397595356,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4869,"Index":861,"Attempt":0,"Launch Time":1427397595347,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595356,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":326694,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4870,"Index":862,"Attempt":0,"Launch Time":1427397595347,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595357,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":318204,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4875,"Index":867,"Attempt":0,"Launch Time":1427397595359,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4872,"Index":864,"Attempt":0,"Launch Time":1427397595350,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595359,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":282982,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4876,"Index":868,"Attempt":0,"Launch Time":1427397595360,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4871,"Index":863,"Attempt":0,"Launch Time":1427397595348,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595361,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":295432,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4877,"Index":869,"Attempt":0,"Launch Time":1427397595364,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4873,"Index":865,"Attempt":0,"Launch Time":1427397595354,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595364,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":3434867,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4878,"Index":870,"Attempt":0,"Launch Time":1427397595364,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4874,"Index":866,"Attempt":0,"Launch Time":1427397595356,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595365,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":261569,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4879,"Index":871,"Attempt":0,"Launch Time":1427397595367,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4876,"Index":868,"Attempt":0,"Launch Time":1427397595360,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595367,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":280841,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4880,"Index":872,"Attempt":0,"Launch Time":1427397595369,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4875,"Index":867,"Attempt":0,"Launch Time":1427397595359,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595369,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":262835,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4881,"Index":873,"Attempt":0,"Launch Time":1427397595374,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4879,"Index":871,"Attempt":0,"Launch Time":1427397595367,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595375,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":350508,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4882,"Index":874,"Attempt":0,"Launch Time":1427397595376,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4878,"Index":870,"Attempt":0,"Launch Time":1427397595364,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595376,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":277408,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4883,"Index":875,"Attempt":0,"Launch Time":1427397595376,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4880,"Index":872,"Attempt":0,"Launch Time":1427397595369,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595376,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":289794,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4884,"Index":876,"Attempt":0,"Launch Time":1427397595377,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4877,"Index":869,"Attempt":0,"Launch Time":1427397595364,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595378,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":580046,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4885,"Index":877,"Attempt":0,"Launch Time":1427397595378,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4864,"Index":856,"Attempt":0,"Launch Time":1427397595341,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595378,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":241883,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4886,"Index":878,"Attempt":0,"Launch Time":1427397595382,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4881,"Index":873,"Attempt":0,"Launch Time":1427397595374,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595382,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":268538,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4887,"Index":879,"Attempt":0,"Launch Time":1427397595382,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4866,"Index":858,"Attempt":0,"Launch Time":1427397595341,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595382,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":41,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":283958,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4888,"Index":880,"Attempt":0,"Launch Time":1427397595385,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4882,"Index":874,"Attempt":0,"Launch Time":1427397595376,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595385,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":280418,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4889,"Index":881,"Attempt":0,"Launch Time":1427397595386,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4883,"Index":875,"Attempt":0,"Launch Time":1427397595376,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595386,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":289528,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4890,"Index":882,"Attempt":0,"Launch Time":1427397595388,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4886,"Index":878,"Attempt":0,"Launch Time":1427397595382,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595388,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":324834,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4891,"Index":883,"Attempt":0,"Launch Time":1427397595390,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595404,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4851,"Index":843,"Attempt":0,"Launch Time":1427397595310,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595390,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":59,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":298334,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4892,"Index":884,"Attempt":0,"Launch Time":1427397595393,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595412,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4856,"Index":848,"Attempt":0,"Launch Time":1427397595318,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595393,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":9,"Executor Run Time":44,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":254064,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4893,"Index":885,"Attempt":0,"Launch Time":1427397595402,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4884,"Index":876,"Attempt":0,"Launch Time":1427397595377,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595402,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":287455,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4894,"Index":886,"Attempt":0,"Launch Time":1427397595403,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595410,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4887,"Index":879,"Attempt":0,"Launch Time":1427397595382,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595403,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":350815,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4895,"Index":887,"Attempt":0,"Launch Time":1427397595404,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4891,"Index":883,"Attempt":0,"Launch Time":1427397595390,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595404,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":423857,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4896,"Index":888,"Attempt":0,"Launch Time":1427397595410,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595418,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4894,"Index":886,"Attempt":0,"Launch Time":1427397595403,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595410,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":285479,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4897,"Index":889,"Attempt":0,"Launch Time":1427397595412,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4898,"Index":890,"Attempt":0,"Launch Time":1427397595412,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4885,"Index":877,"Attempt":0,"Launch Time":1427397595378,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595412,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":14,"Executor Run Time":13,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":361470,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4892,"Index":884,"Attempt":0,"Launch Time":1427397595393,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595412,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":392519,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4899,"Index":891,"Attempt":0,"Launch Time":1427397595418,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4896,"Index":888,"Attempt":0,"Launch Time":1427397595410,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595418,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":324227,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4900,"Index":892,"Attempt":0,"Launch Time":1427397595434,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4898,"Index":890,"Attempt":0,"Launch Time":1427397595412,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595434,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":10685409,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4901,"Index":893,"Attempt":0,"Launch Time":1427397595435,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4902,"Index":894,"Attempt":0,"Launch Time":1427397595435,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4893,"Index":885,"Attempt":0,"Launch Time":1427397595402,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595435,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":26,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":541919,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4897,"Index":889,"Attempt":0,"Launch Time":1427397595412,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595435,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":19,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":324234,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4903,"Index":895,"Attempt":0,"Launch Time":1427397595435,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4899,"Index":891,"Attempt":0,"Launch Time":1427397595418,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595435,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":10763889,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4904,"Index":896,"Attempt":0,"Launch Time":1427397595440,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4890,"Index":882,"Attempt":0,"Launch Time":1427397595388,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595440,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":17,"Executor Run Time":29,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":11359990,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4905,"Index":897,"Attempt":0,"Launch Time":1427397595441,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4895,"Index":887,"Attempt":0,"Launch Time":1427397595404,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595441,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":31,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":284904,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4906,"Index":898,"Attempt":0,"Launch Time":1427397595441,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4901,"Index":893,"Attempt":0,"Launch Time":1427397595435,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595441,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":292045,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4907,"Index":899,"Attempt":0,"Launch Time":1427397595442,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4903,"Index":895,"Attempt":0,"Launch Time":1427397595435,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595442,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":279689,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4908,"Index":900,"Attempt":0,"Launch Time":1427397595444,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4888,"Index":880,"Attempt":0,"Launch Time":1427397595385,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595444,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":48,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":269019,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4909,"Index":901,"Attempt":0,"Launch Time":1427397595444,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4902,"Index":894,"Attempt":0,"Launch Time":1427397595435,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595444,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":301232,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4910,"Index":902,"Attempt":0,"Launch Time":1427397595447,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4900,"Index":892,"Attempt":0,"Launch Time":1427397595434,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595447,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1280351,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4911,"Index":903,"Attempt":0,"Launch Time":1427397595447,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4905,"Index":897,"Attempt":0,"Launch Time":1427397595441,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595447,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":296378,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4912,"Index":904,"Attempt":0,"Launch Time":1427397595449,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4913,"Index":905,"Attempt":0,"Launch Time":1427397595449,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4907,"Index":899,"Attempt":0,"Launch Time":1427397595442,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595449,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":305312,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4906,"Index":898,"Attempt":0,"Launch Time":1427397595441,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595449,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":323382,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4914,"Index":906,"Attempt":0,"Launch Time":1427397595451,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4908,"Index":900,"Attempt":0,"Launch Time":1427397595444,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595451,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":312028,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4915,"Index":907,"Attempt":0,"Launch Time":1427397595454,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4911,"Index":903,"Attempt":0,"Launch Time":1427397595447,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595454,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9608,"Shuffle Write Time":354831,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4916,"Index":908,"Attempt":0,"Launch Time":1427397595457,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4917,"Index":909,"Attempt":0,"Launch Time":1427397595457,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4912,"Index":904,"Attempt":0,"Launch Time":1427397595449,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595457,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":379803,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4909,"Index":901,"Attempt":0,"Launch Time":1427397595444,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595457,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":5,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":316990,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4918,"Index":910,"Attempt":0,"Launch Time":1427397595458,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4913,"Index":905,"Attempt":0,"Launch Time":1427397595449,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595458,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1321660,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4919,"Index":911,"Attempt":0,"Launch Time":1427397595460,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4914,"Index":906,"Attempt":0,"Launch Time":1427397595451,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595460,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":315089,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4920,"Index":912,"Attempt":0,"Launch Time":1427397595461,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4915,"Index":907,"Attempt":0,"Launch Time":1427397595454,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595461,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":276366,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4921,"Index":913,"Attempt":0,"Launch Time":1427397595463,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4910,"Index":902,"Attempt":0,"Launch Time":1427397595447,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595463,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":302635,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4922,"Index":914,"Attempt":0,"Launch Time":1427397595466,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4917,"Index":909,"Attempt":0,"Launch Time":1427397595457,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595466,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":293116,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4923,"Index":915,"Attempt":0,"Launch Time":1427397595467,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4920,"Index":912,"Attempt":0,"Launch Time":1427397595461,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595467,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":270318,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4924,"Index":916,"Attempt":0,"Launch Time":1427397595469,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4921,"Index":913,"Attempt":0,"Launch Time":1427397595463,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595469,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":252484,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4925,"Index":917,"Attempt":0,"Launch Time":1427397595469,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4919,"Index":911,"Attempt":0,"Launch Time":1427397595460,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595470,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":348723,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4926,"Index":918,"Attempt":0,"Launch Time":1427397595471,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4918,"Index":910,"Attempt":0,"Launch Time":1427397595458,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595472,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1728264,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4927,"Index":919,"Attempt":0,"Launch Time":1427397595474,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4923,"Index":915,"Attempt":0,"Launch Time":1427397595467,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595474,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":293835,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4928,"Index":920,"Attempt":0,"Launch Time":1427397595475,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4924,"Index":916,"Attempt":0,"Launch Time":1427397595469,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595475,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":310321,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4929,"Index":921,"Attempt":0,"Launch Time":1427397595476,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4925,"Index":917,"Attempt":0,"Launch Time":1427397595469,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595476,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":304147,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4930,"Index":922,"Attempt":0,"Launch Time":1427397595480,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4927,"Index":919,"Attempt":0,"Launch Time":1427397595474,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595480,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":272914,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4931,"Index":923,"Attempt":0,"Launch Time":1427397595482,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4928,"Index":920,"Attempt":0,"Launch Time":1427397595475,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595482,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":304796,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4932,"Index":924,"Attempt":0,"Launch Time":1427397595482,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4926,"Index":918,"Attempt":0,"Launch Time":1427397595471,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595482,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":304985,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4933,"Index":925,"Attempt":0,"Launch Time":1427397595484,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4929,"Index":921,"Attempt":0,"Launch Time":1427397595476,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595484,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":285552,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4934,"Index":926,"Attempt":0,"Launch Time":1427397595485,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4922,"Index":914,"Attempt":0,"Launch Time":1427397595466,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595485,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":7,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":312232,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4935,"Index":927,"Attempt":0,"Launch Time":1427397595487,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4916,"Index":908,"Attempt":0,"Launch Time":1427397595457,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595487,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":6,"Executor Run Time":24,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":317529,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4936,"Index":928,"Attempt":0,"Launch Time":1427397595487,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4930,"Index":922,"Attempt":0,"Launch Time":1427397595480,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595487,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":365451,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4937,"Index":929,"Attempt":0,"Launch Time":1427397595488,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4931,"Index":923,"Attempt":0,"Launch Time":1427397595482,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595488,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":266929,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4938,"Index":930,"Attempt":0,"Launch Time":1427397595494,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4889,"Index":881,"Attempt":0,"Launch Time":1427397595386,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595495,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":29,"Executor Run Time":72,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":8656692,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4939,"Index":931,"Attempt":0,"Launch Time":1427397595497,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4932,"Index":924,"Attempt":0,"Launch Time":1427397595482,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595497,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":451659,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4940,"Index":932,"Attempt":0,"Launch Time":1427397595499,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4936,"Index":928,"Attempt":0,"Launch Time":1427397595487,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595499,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":273567,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4941,"Index":933,"Attempt":0,"Launch Time":1427397595500,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4937,"Index":929,"Attempt":0,"Launch Time":1427397595488,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595500,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":306672,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4942,"Index":934,"Attempt":0,"Launch Time":1427397595501,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4943,"Index":935,"Attempt":0,"Launch Time":1427397595502,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4933,"Index":925,"Attempt":0,"Launch Time":1427397595484,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595502,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":318912,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4934,"Index":926,"Attempt":0,"Launch Time":1427397595485,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595502,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":12,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":302838,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4944,"Index":936,"Attempt":0,"Launch Time":1427397595505,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4940,"Index":932,"Attempt":0,"Launch Time":1427397595499,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595505,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":332066,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4945,"Index":937,"Attempt":0,"Launch Time":1427397595506,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4939,"Index":931,"Attempt":0,"Launch Time":1427397595497,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595506,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":588088,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4946,"Index":938,"Attempt":0,"Launch Time":1427397595510,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4943,"Index":935,"Attempt":0,"Launch Time":1427397595502,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595510,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":984894,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4947,"Index":939,"Attempt":0,"Launch Time":1427397595511,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4942,"Index":934,"Attempt":0,"Launch Time":1427397595501,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595511,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":295492,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4948,"Index":940,"Attempt":0,"Launch Time":1427397595515,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4938,"Index":930,"Attempt":0,"Launch Time":1427397595494,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595515,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":313262,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4949,"Index":941,"Attempt":0,"Launch Time":1427397595516,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4904,"Index":896,"Attempt":0,"Launch Time":1427397595440,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595516,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":9,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":311612,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4950,"Index":942,"Attempt":0,"Launch Time":1427397595519,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4946,"Index":938,"Attempt":0,"Launch Time":1427397595510,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595519,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":269914,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4951,"Index":943,"Attempt":0,"Launch Time":1427397595521,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4935,"Index":927,"Attempt":0,"Launch Time":1427397595487,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595521,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":32,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":388904,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4952,"Index":944,"Attempt":0,"Launch Time":1427397595523,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4948,"Index":940,"Attempt":0,"Launch Time":1427397595515,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595523,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":276925,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4953,"Index":945,"Attempt":0,"Launch Time":1427397595523,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4949,"Index":941,"Attempt":0,"Launch Time":1427397595516,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595523,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":331196,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4954,"Index":946,"Attempt":0,"Launch Time":1427397595528,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4951,"Index":943,"Attempt":0,"Launch Time":1427397595521,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595529,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":318937,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4955,"Index":947,"Attempt":0,"Launch Time":1427397595530,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4952,"Index":944,"Attempt":0,"Launch Time":1427397595523,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595530,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":324905,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4956,"Index":948,"Attempt":0,"Launch Time":1427397595532,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4953,"Index":945,"Attempt":0,"Launch Time":1427397595523,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595532,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":337478,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4957,"Index":949,"Attempt":0,"Launch Time":1427397595538,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4944,"Index":936,"Attempt":0,"Launch Time":1427397595505,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595538,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":32,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1067240,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4958,"Index":950,"Attempt":0,"Launch Time":1427397595540,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4955,"Index":947,"Attempt":0,"Launch Time":1427397595530,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595540,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":392630,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4959,"Index":951,"Attempt":0,"Launch Time":1427397595541,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4956,"Index":948,"Attempt":0,"Launch Time":1427397595532,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595541,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":310096,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4960,"Index":952,"Attempt":0,"Launch Time":1427397595542,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4954,"Index":946,"Attempt":0,"Launch Time":1427397595528,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595542,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":766140,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4961,"Index":953,"Attempt":0,"Launch Time":1427397595542,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4950,"Index":942,"Attempt":0,"Launch Time":1427397595519,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595542,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":720164,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4962,"Index":954,"Attempt":0,"Launch Time":1427397595546,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4957,"Index":949,"Attempt":0,"Launch Time":1427397595538,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595546,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":309977,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4963,"Index":955,"Attempt":0,"Launch Time":1427397595548,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4959,"Index":951,"Attempt":0,"Launch Time":1427397595541,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595548,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":283008,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4964,"Index":956,"Attempt":0,"Launch Time":1427397595548,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4960,"Index":952,"Attempt":0,"Launch Time":1427397595542,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595548,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":280324,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4965,"Index":957,"Attempt":0,"Launch Time":1427397595548,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4958,"Index":950,"Attempt":0,"Launch Time":1427397595540,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595549,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":267444,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4966,"Index":958,"Attempt":0,"Launch Time":1427397595549,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4961,"Index":953,"Attempt":0,"Launch Time":1427397595542,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595549,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":269989,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4967,"Index":959,"Attempt":0,"Launch Time":1427397595551,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4945,"Index":937,"Attempt":0,"Launch Time":1427397595506,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595551,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":44,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":294554,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4968,"Index":960,"Attempt":0,"Launch Time":1427397595552,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4962,"Index":954,"Attempt":0,"Launch Time":1427397595546,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595552,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":286139,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4969,"Index":961,"Attempt":0,"Launch Time":1427397595554,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4963,"Index":955,"Attempt":0,"Launch Time":1427397595548,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595554,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":287846,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4970,"Index":962,"Attempt":0,"Launch Time":1427397595554,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4964,"Index":956,"Attempt":0,"Launch Time":1427397595548,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595555,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":284064,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4971,"Index":963,"Attempt":0,"Launch Time":1427397595555,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4965,"Index":957,"Attempt":0,"Launch Time":1427397595548,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595555,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":276020,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4972,"Index":964,"Attempt":0,"Launch Time":1427397595557,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4966,"Index":958,"Attempt":0,"Launch Time":1427397595549,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595557,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":255942,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4973,"Index":965,"Attempt":0,"Launch Time":1427397595559,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4968,"Index":960,"Attempt":0,"Launch Time":1427397595552,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595559,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":288473,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4974,"Index":966,"Attempt":0,"Launch Time":1427397595560,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4969,"Index":961,"Attempt":0,"Launch Time":1427397595554,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595560,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":252827,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4975,"Index":967,"Attempt":0,"Launch Time":1427397595560,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4970,"Index":962,"Attempt":0,"Launch Time":1427397595554,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595561,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":295502,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4976,"Index":968,"Attempt":0,"Launch Time":1427397595563,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4971,"Index":963,"Attempt":0,"Launch Time":1427397595555,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595563,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":355260,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4977,"Index":969,"Attempt":0,"Launch Time":1427397595565,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4973,"Index":965,"Attempt":0,"Launch Time":1427397595559,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595565,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":256558,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4978,"Index":970,"Attempt":0,"Launch Time":1427397595566,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4979,"Index":971,"Attempt":0,"Launch Time":1427397595566,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4972,"Index":964,"Attempt":0,"Launch Time":1427397595557,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595566,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":346828,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4974,"Index":966,"Attempt":0,"Launch Time":1427397595560,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595566,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":278725,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4980,"Index":972,"Attempt":0,"Launch Time":1427397595568,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4975,"Index":967,"Attempt":0,"Launch Time":1427397595560,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595568,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":270771,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4981,"Index":973,"Attempt":0,"Launch Time":1427397595568,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4976,"Index":968,"Attempt":0,"Launch Time":1427397595563,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595569,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":231183,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4982,"Index":974,"Attempt":0,"Launch Time":1427397595569,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4967,"Index":959,"Attempt":0,"Launch Time":1427397595551,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595569,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":270758,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4983,"Index":975,"Attempt":0,"Launch Time":1427397595578,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4978,"Index":970,"Attempt":0,"Launch Time":1427397595566,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595578,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":282320,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4984,"Index":976,"Attempt":0,"Launch Time":1427397595580,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4977,"Index":969,"Attempt":0,"Launch Time":1427397595565,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595580,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":268832,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4985,"Index":977,"Attempt":0,"Launch Time":1427397595582,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4947,"Index":939,"Attempt":0,"Launch Time":1427397595511,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595582,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":6,"Executor Run Time":32,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":310797,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4986,"Index":978,"Attempt":0,"Launch Time":1427397595582,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4980,"Index":972,"Attempt":0,"Launch Time":1427397595568,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595583,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":371070,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4987,"Index":979,"Attempt":0,"Launch Time":1427397595584,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4981,"Index":973,"Attempt":0,"Launch Time":1427397595568,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595584,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1284696,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4988,"Index":980,"Attempt":0,"Launch Time":1427397595587,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4989,"Index":981,"Attempt":0,"Launch Time":1427397595587,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4984,"Index":976,"Attempt":0,"Launch Time":1427397595580,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595587,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":338724,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4979,"Index":971,"Attempt":0,"Launch Time":1427397595566,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595587,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":354911,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4990,"Index":982,"Attempt":0,"Launch Time":1427397595588,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4983,"Index":975,"Attempt":0,"Launch Time":1427397595578,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595588,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":347287,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4991,"Index":983,"Attempt":0,"Launch Time":1427397595590,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4985,"Index":977,"Attempt":0,"Launch Time":1427397595582,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595591,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":416744,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4992,"Index":984,"Attempt":0,"Launch Time":1427397595592,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4987,"Index":979,"Attempt":0,"Launch Time":1427397595584,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595593,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":330947,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4993,"Index":985,"Attempt":0,"Launch Time":1427397595593,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4994,"Index":986,"Attempt":0,"Launch Time":1427397595593,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4982,"Index":974,"Attempt":0,"Launch Time":1427397595569,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595594,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":308913,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4989,"Index":981,"Attempt":0,"Launch Time":1427397595587,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595594,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":276534,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4995,"Index":987,"Attempt":0,"Launch Time":1427397595600,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4990,"Index":982,"Attempt":0,"Launch Time":1427397595588,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595601,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":306233,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4996,"Index":988,"Attempt":0,"Launch Time":1427397595601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4992,"Index":984,"Attempt":0,"Launch Time":1427397595592,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595601,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":274530,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4997,"Index":989,"Attempt":0,"Launch Time":1427397595601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4941,"Index":933,"Attempt":0,"Launch Time":1427397595500,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595601,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":97,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":40068811,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4998,"Index":990,"Attempt":0,"Launch Time":1427397595601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4999,"Index":991,"Attempt":0,"Launch Time":1427397595601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4994,"Index":986,"Attempt":0,"Launch Time":1427397595593,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595602,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":328211,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4988,"Index":980,"Attempt":0,"Launch Time":1427397595587,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595602,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":264650,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":5000,"Index":992,"Attempt":0,"Launch Time":1427397595602,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4991,"Index":983,"Attempt":0,"Launch Time":1427397595590,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595602,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":273561,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":5001,"Index":993,"Attempt":0,"Launch Time":1427397595603,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4986,"Index":978,"Attempt":0,"Launch Time":1427397595582,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595604,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":20,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":368979,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":5002,"Index":994,"Attempt":0,"Launch Time":1427397595608,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4995,"Index":987,"Attempt":0,"Launch Time":1427397595600,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595608,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":299626,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":5003,"Index":995,"Attempt":0,"Launch Time":1427397595608,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4999,"Index":991,"Attempt":0,"Launch Time":1427397595601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595608,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":308770,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":5004,"Index":996,"Attempt":0,"Launch Time":1427397595609,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4998,"Index":990,"Attempt":0,"Launch Time":1427397595601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595609,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":1517002,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":5005,"Index":997,"Attempt":0,"Launch Time":1427397595609,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4996,"Index":988,"Attempt":0,"Launch Time":1427397595601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595609,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":294365,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":5006,"Index":998,"Attempt":0,"Launch Time":1427397595611,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5000,"Index":992,"Attempt":0,"Launch Time":1427397595602,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595611,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":292874,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":5007,"Index":999,"Attempt":0,"Launch Time":1427397595611,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4997,"Index":989,"Attempt":0,"Launch Time":1427397595601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595612,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":319333,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5001,"Index":993,"Attempt":0,"Launch Time":1427397595603,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595615,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":276786,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5003,"Index":995,"Attempt":0,"Launch Time":1427397595608,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595617,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":273428,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5004,"Index":996,"Attempt":0,"Launch Time":1427397595609,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595617,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":273868,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5006,"Index":998,"Attempt":0,"Launch Time":1427397595611,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595618,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":275973,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5007,"Index":999,"Attempt":0,"Launch Time":1427397595611,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595618,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":263073,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4993,"Index":985,"Attempt":0,"Launch Time":1427397595593,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595620,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":23,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":2842856,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5005,"Index":997,"Attempt":0,"Launch Time":1427397595609,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595621,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":287183,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5002,"Index":994,"Attempt":0,"Launch Time":1427397595608,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595623,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":297691,"Shuffle Records Written":100}}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":16,"Stage Attempt ID":0,"Stage Name":"map at :21","Number of Tasks":1000,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:287)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:21)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:28)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:30)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.(:32)\n$line15.$read$$iwC$$iwC$$iwC.(:34)\n$line15.$read$$iwC$$iwC.(:36)\n$line15.$read$$iwC.(:38)\n$line15.$read.(:40)\n$line15.$read$.(:44)\n$line15.$read$.()\n$line15.$eval$.(:7)\n$line15.$eval$.()\n$line15.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Submission Time":1427397593694,"Completion Time":1427397595623,"Accumulables":[]}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":17,"Stage Attempt ID":0,"Stage Name":"next at :26","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":50,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"scala.collection.Iterator$$anon$13.next(Iterator.scala:372)\n$line36.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line36.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:31)\n$line36.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:33)\n$line36.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:35)\n$line36.$read$$iwC$$iwC$$iwC$$iwC.(:37)\n$line36.$read$$iwC$$iwC$$iwC.(:39)\n$line36.$read$$iwC$$iwC.(:41)\n$line36.$read$$iwC.(:43)\n$line36.$read.(:45)\n$line36.$read$.(:49)\n$line36.$read$.()\n$line36.$eval$.(:7)\n$line36.$eval$.()\n$line36.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":17,"Stage Attempt ID":0,"Task Info":{"Task ID":5008,"Index":0,"Attempt":0,"Launch Time":1427397595627,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":17,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5008,"Index":0,"Attempt":0,"Launch Time":1427397595627,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595702,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":74,"Result Size":1060,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":1000,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":192025,"Total Records Read":2000}}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":17,"Stage Attempt ID":0,"Stage Name":"next at :26","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":50,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"scala.collection.Iterator$$anon$13.next(Iterator.scala:372)\n$line36.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line36.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:31)\n$line36.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:33)\n$line36.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:35)\n$line36.$read$$iwC$$iwC$$iwC$$iwC.(:37)\n$line36.$read$$iwC$$iwC$$iwC.(:39)\n$line36.$read$$iwC$$iwC.(:41)\n$line36.$read$$iwC.(:43)\n$line36.$read.(:45)\n$line36.$read$.(:49)\n$line36.$read$.()\n$line36.$eval$.(:7)\n$line36.$eval$.()\n$line36.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Submission Time":1427397595627,"Completion Time":1427397595702,"Accumulables":[]}} -{"Event":"SparkListenerJobEnd","Job ID":8,"Completion Time":1427397595702,"Job Result":{"Result":"JobSucceeded"}} -{"Event":"SparkListenerJobStart","Job ID":9,"Submission Time":1427397597114,"Stage Infos":[{"Stage ID":19,"Stage Attempt ID":0,"Stage Name":"next at :26","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":50,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"scala.collection.Iterator$$anon$13.next(Iterator.scala:372)\n$line38.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line38.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:31)\n$line38.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:33)\n$line38.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:35)\n$line38.$read$$iwC$$iwC$$iwC$$iwC.(:37)\n$line38.$read$$iwC$$iwC$$iwC.(:39)\n$line38.$read$$iwC$$iwC.(:41)\n$line38.$read$$iwC.(:43)\n$line38.$read.(:45)\n$line38.$read$.(:49)\n$line38.$read$.()\n$line38.$eval$.(:7)\n$line38.$eval$.()\n$line38.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]},{"Stage ID":18,"Stage Attempt ID":0,"Stage Name":"map at :21","Number of Tasks":1000,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:287)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:21)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:28)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:30)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.(:32)\n$line15.$read$$iwC$$iwC$$iwC.(:34)\n$line15.$read$$iwC$$iwC.(:36)\n$line15.$read$$iwC.(:38)\n$line15.$read.(:40)\n$line15.$read$.(:44)\n$line15.$read$.()\n$line15.$eval$.(:7)\n$line15.$eval$.()\n$line15.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]}],"Stage IDs":[19,18]} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":19,"Stage Attempt ID":0,"Stage Name":"next at :26","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":50,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"scala.collection.Iterator$$anon$13.next(Iterator.scala:372)\n$line38.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line38.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:31)\n$line38.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:33)\n$line38.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:35)\n$line38.$read$$iwC$$iwC$$iwC$$iwC.(:37)\n$line38.$read$$iwC$$iwC$$iwC.(:39)\n$line38.$read$$iwC$$iwC.(:41)\n$line38.$read$$iwC.(:43)\n$line38.$read.(:45)\n$line38.$read$.(:49)\n$line38.$read$.()\n$line38.$eval$.(:7)\n$line38.$eval$.()\n$line38.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":19,"Stage Attempt ID":0,"Task Info":{"Task ID":5009,"Index":0,"Attempt":0,"Launch Time":1427397597116,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":19,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5009,"Index":0,"Attempt":0,"Launch Time":1427397597116,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397597205,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":88,"Result Size":1060,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":1000,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":192022,"Total Records Read":2000}}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":19,"Stage Attempt ID":0,"Stage Name":"next at :26","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":50,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"scala.collection.Iterator$$anon$13.next(Iterator.scala:372)\n$line38.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line38.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:31)\n$line38.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:33)\n$line38.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:35)\n$line38.$read$$iwC$$iwC$$iwC$$iwC.(:37)\n$line38.$read$$iwC$$iwC$$iwC.(:39)\n$line38.$read$$iwC$$iwC.(:41)\n$line38.$read$$iwC.(:43)\n$line38.$read.(:45)\n$line38.$read$.(:49)\n$line38.$read$.()\n$line38.$eval$.(:7)\n$line38.$eval$.()\n$line38.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Submission Time":1427397597116,"Completion Time":1427397597205,"Accumulables":[]}} -{"Event":"SparkListenerJobEnd","Job ID":9,"Completion Time":1427397597205,"Job Result":{"Result":"JobSucceeded"}} -{"Event":"SparkListenerJobStart","Job ID":10,"Submission Time":1427397599546,"Stage Infos":[{"Stage ID":20,"Stage Attempt ID":0,"Stage Name":"map at :21","Number of Tasks":1000,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:287)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:21)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:28)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:30)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.(:32)\n$line15.$read$$iwC$$iwC$$iwC.(:34)\n$line15.$read$$iwC$$iwC.(:36)\n$line15.$read$$iwC.(:38)\n$line15.$read.(:40)\n$line15.$read$.(:44)\n$line15.$read$.()\n$line15.$eval$.(:7)\n$line15.$eval$.()\n$line15.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]},{"Stage ID":21,"Stage Attempt ID":0,"Stage Name":"next at :26","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":50,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"scala.collection.Iterator$$anon$13.next(Iterator.scala:372)\n$line40.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line40.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:31)\n$line40.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:33)\n$line40.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:35)\n$line40.$read$$iwC$$iwC$$iwC$$iwC.(:37)\n$line40.$read$$iwC$$iwC$$iwC.(:39)\n$line40.$read$$iwC$$iwC.(:41)\n$line40.$read$$iwC.(:43)\n$line40.$read.(:45)\n$line40.$read$.(:49)\n$line40.$read$.()\n$line40.$eval$.(:7)\n$line40.$eval$.()\n$line40.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]}],"Stage IDs":[20,21]} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":20,"Stage Attempt ID":0,"Stage Name":"map at :21","Number of Tasks":1000,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:287)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:21)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:28)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:30)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.(:32)\n$line15.$read$$iwC$$iwC$$iwC.(:34)\n$line15.$read$$iwC$$iwC.(:36)\n$line15.$read$$iwC.(:38)\n$line15.$read.(:40)\n$line15.$read$.(:44)\n$line15.$read$.()\n$line15.$eval$.(:7)\n$line15.$eval$.()\n$line15.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5010,"Index":0,"Attempt":0,"Launch Time":1427397599571,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5011,"Index":1,"Attempt":0,"Launch Time":1427397599571,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5012,"Index":2,"Attempt":0,"Launch Time":1427397599571,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5013,"Index":3,"Attempt":0,"Launch Time":1427397599571,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5014,"Index":4,"Attempt":0,"Launch Time":1427397599571,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5015,"Index":5,"Attempt":0,"Launch Time":1427397599571,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5016,"Index":6,"Attempt":0,"Launch Time":1427397599571,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5017,"Index":7,"Attempt":0,"Launch Time":1427397599571,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5018,"Index":8,"Attempt":0,"Launch Time":1427397599579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5019,"Index":9,"Attempt":0,"Launch Time":1427397599579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5020,"Index":10,"Attempt":0,"Launch Time":1427397599579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5021,"Index":11,"Attempt":0,"Launch Time":1427397599579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5022,"Index":12,"Attempt":0,"Launch Time":1427397599579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5023,"Index":13,"Attempt":0,"Launch Time":1427397599579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5011,"Index":1,"Attempt":0,"Launch Time":1427397599571,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599580,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":271674,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5012,"Index":2,"Attempt":0,"Launch Time":1427397599571,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599585,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":274899,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5016,"Index":6,"Attempt":0,"Launch Time":1427397599571,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599586,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":275977,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5024,"Index":14,"Attempt":0,"Launch Time":1427397599586,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5025,"Index":15,"Attempt":0,"Launch Time":1427397599586,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5026,"Index":16,"Attempt":0,"Launch Time":1427397599586,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5027,"Index":17,"Attempt":0,"Launch Time":1427397599586,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5019,"Index":9,"Attempt":0,"Launch Time":1427397599579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599586,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":319028,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5020,"Index":10,"Attempt":0,"Launch Time":1427397599579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599586,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":306724,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5014,"Index":4,"Attempt":0,"Launch Time":1427397599571,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599586,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":316561,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5021,"Index":11,"Attempt":0,"Launch Time":1427397599579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599587,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":280729,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5017,"Index":7,"Attempt":0,"Launch Time":1427397599571,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599591,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":267412,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5015,"Index":5,"Attempt":0,"Launch Time":1427397599571,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599592,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":269443,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5028,"Index":18,"Attempt":0,"Launch Time":1427397599592,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5029,"Index":19,"Attempt":0,"Launch Time":1427397599593,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5030,"Index":20,"Attempt":0,"Launch Time":1427397599593,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5031,"Index":21,"Attempt":0,"Launch Time":1427397599593,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5032,"Index":22,"Attempt":0,"Launch Time":1427397599593,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5033,"Index":23,"Attempt":0,"Launch Time":1427397599593,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5034,"Index":24,"Attempt":0,"Launch Time":1427397599594,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5010,"Index":0,"Attempt":0,"Launch Time":1427397599571,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599594,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":318464,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5026,"Index":16,"Attempt":0,"Launch Time":1427397599586,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599594,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":247775,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5022,"Index":12,"Attempt":0,"Launch Time":1427397599579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599594,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":6,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":273837,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5027,"Index":17,"Attempt":0,"Launch Time":1427397599586,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599594,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":289503,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5024,"Index":14,"Attempt":0,"Launch Time":1427397599586,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599596,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":266308,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5023,"Index":13,"Attempt":0,"Launch Time":1427397599579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599596,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":6,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":282108,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5018,"Index":8,"Attempt":0,"Launch Time":1427397599579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599598,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":310279,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5013,"Index":3,"Attempt":0,"Launch Time":1427397599571,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599600,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":315224,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5035,"Index":25,"Attempt":0,"Launch Time":1427397599600,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5036,"Index":26,"Attempt":0,"Launch Time":1427397599600,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5037,"Index":27,"Attempt":0,"Launch Time":1427397599601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5038,"Index":28,"Attempt":0,"Launch Time":1427397599601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5039,"Index":29,"Attempt":0,"Launch Time":1427397599601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5040,"Index":30,"Attempt":0,"Launch Time":1427397599601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5032,"Index":22,"Attempt":0,"Launch Time":1427397599593,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599601,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":345941,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5033,"Index":23,"Attempt":0,"Launch Time":1427397599593,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599601,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":301382,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5034,"Index":24,"Attempt":0,"Launch Time":1427397599594,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599602,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":340547,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5025,"Index":15,"Attempt":0,"Launch Time":1427397599586,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599602,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":330830,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5041,"Index":31,"Attempt":0,"Launch Time":1427397599602,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5042,"Index":32,"Attempt":0,"Launch Time":1427397599602,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5030,"Index":20,"Attempt":0,"Launch Time":1427397599593,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599602,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":367295,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5031,"Index":21,"Attempt":0,"Launch Time":1427397599593,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599603,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":493701,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5043,"Index":33,"Attempt":0,"Launch Time":1427397599607,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5044,"Index":34,"Attempt":0,"Launch Time":1427397599607,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5035,"Index":25,"Attempt":0,"Launch Time":1427397599600,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599608,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":324374,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5036,"Index":26,"Attempt":0,"Launch Time":1427397599600,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599608,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":321083,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5045,"Index":35,"Attempt":0,"Launch Time":1427397599608,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5046,"Index":36,"Attempt":0,"Launch Time":1427397599609,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5047,"Index":37,"Attempt":0,"Launch Time":1427397599609,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5048,"Index":38,"Attempt":0,"Launch Time":1427397599609,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5029,"Index":19,"Attempt":0,"Launch Time":1427397599593,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599610,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":339081,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5037,"Index":27,"Attempt":0,"Launch Time":1427397599601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599610,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":318662,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5039,"Index":29,"Attempt":0,"Launch Time":1427397599601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599610,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":287042,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5049,"Index":39,"Attempt":0,"Launch Time":1427397599611,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5028,"Index":18,"Attempt":0,"Launch Time":1427397599592,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599612,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":355545,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5042,"Index":32,"Attempt":0,"Launch Time":1427397599602,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599612,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":297534,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5038,"Index":28,"Attempt":0,"Launch Time":1427397599601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599613,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":334376,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5040,"Index":30,"Attempt":0,"Launch Time":1427397599601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599615,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":332938,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5050,"Index":40,"Attempt":0,"Launch Time":1427397599615,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5051,"Index":41,"Attempt":0,"Launch Time":1427397599615,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5052,"Index":42,"Attempt":0,"Launch Time":1427397599615,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5053,"Index":43,"Attempt":0,"Launch Time":1427397599616,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5041,"Index":31,"Attempt":0,"Launch Time":1427397599602,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599616,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":255099,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5044,"Index":34,"Attempt":0,"Launch Time":1427397599607,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599616,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":279851,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5043,"Index":33,"Attempt":0,"Launch Time":1427397599607,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599616,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":265929,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5045,"Index":35,"Attempt":0,"Launch Time":1427397599608,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599616,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":278965,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5054,"Index":44,"Attempt":0,"Launch Time":1427397599616,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5055,"Index":45,"Attempt":0,"Launch Time":1427397599617,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5056,"Index":46,"Attempt":0,"Launch Time":1427397599617,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5046,"Index":36,"Attempt":0,"Launch Time":1427397599609,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599617,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":340518,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5048,"Index":38,"Attempt":0,"Launch Time":1427397599609,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599617,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":265447,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5047,"Index":37,"Attempt":0,"Launch Time":1427397599609,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599618,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":402877,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5057,"Index":47,"Attempt":0,"Launch Time":1427397599639,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599646,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5049,"Index":39,"Attempt":0,"Launch Time":1427397599611,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599639,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":23,"Result Size":930,"JVM GC Time":16,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":304745,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5058,"Index":48,"Attempt":0,"Launch Time":1427397599640,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599691,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5059,"Index":49,"Attempt":0,"Launch Time":1427397599641,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599651,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5050,"Index":40,"Attempt":0,"Launch Time":1427397599615,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599641,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":24,"Result Size":930,"JVM GC Time":16,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":302371,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5052,"Index":42,"Attempt":0,"Launch Time":1427397599615,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599641,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":24,"Result Size":930,"JVM GC Time":16,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":962033,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5060,"Index":50,"Attempt":0,"Launch Time":1427397599642,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599666,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5061,"Index":51,"Attempt":0,"Launch Time":1427397599642,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599651,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5051,"Index":41,"Attempt":0,"Launch Time":1427397599615,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599642,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":25,"Result Size":930,"JVM GC Time":16,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":296177,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5062,"Index":52,"Attempt":0,"Launch Time":1427397599642,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599661,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5056,"Index":46,"Attempt":0,"Launch Time":1427397599617,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599642,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":17,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":306911,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5055,"Index":45,"Attempt":0,"Launch Time":1427397599617,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599643,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":25,"Result Size":930,"JVM GC Time":16,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":307821,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5063,"Index":53,"Attempt":0,"Launch Time":1427397599643,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599651,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5053,"Index":43,"Attempt":0,"Launch Time":1427397599616,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599643,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":26,"Result Size":930,"JVM GC Time":16,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":310335,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5064,"Index":54,"Attempt":0,"Launch Time":1427397599646,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599660,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5057,"Index":47,"Attempt":0,"Launch Time":1427397599639,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599646,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":274191,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5065,"Index":55,"Attempt":0,"Launch Time":1427397599650,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599660,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5066,"Index":56,"Attempt":0,"Launch Time":1427397599650,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599664,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5067,"Index":57,"Attempt":0,"Launch Time":1427397599650,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599660,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5061,"Index":51,"Attempt":0,"Launch Time":1427397599642,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599651,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":313090,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5059,"Index":49,"Attempt":0,"Launch Time":1427397599641,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599651,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":300675,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5063,"Index":53,"Attempt":0,"Launch Time":1427397599643,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599651,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":312481,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5068,"Index":58,"Attempt":0,"Launch Time":1427397599659,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599667,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5065,"Index":55,"Attempt":0,"Launch Time":1427397599650,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599660,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":670271,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5069,"Index":59,"Attempt":0,"Launch Time":1427397599660,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599668,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5070,"Index":60,"Attempt":0,"Launch Time":1427397599660,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599669,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5064,"Index":54,"Attempt":0,"Launch Time":1427397599646,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599660,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":296976,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5067,"Index":57,"Attempt":0,"Launch Time":1427397599650,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599660,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":678632,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5071,"Index":61,"Attempt":0,"Launch Time":1427397599661,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599674,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5062,"Index":52,"Attempt":0,"Launch Time":1427397599642,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599661,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1572828,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5072,"Index":62,"Attempt":0,"Launch Time":1427397599664,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599670,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5066,"Index":56,"Attempt":0,"Launch Time":1427397599650,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599664,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1240309,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5073,"Index":63,"Attempt":0,"Launch Time":1427397599666,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599681,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5060,"Index":50,"Attempt":0,"Launch Time":1427397599642,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599666,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":24,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":812196,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5074,"Index":64,"Attempt":0,"Launch Time":1427397599667,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599675,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5068,"Index":58,"Attempt":0,"Launch Time":1427397599659,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599667,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":302888,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5075,"Index":65,"Attempt":0,"Launch Time":1427397599668,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599679,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5069,"Index":59,"Attempt":0,"Launch Time":1427397599660,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599668,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9610,"Shuffle Write Time":383153,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5076,"Index":66,"Attempt":0,"Launch Time":1427397599669,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599687,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5070,"Index":60,"Attempt":0,"Launch Time":1427397599660,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599669,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":310480,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5077,"Index":67,"Attempt":0,"Launch Time":1427397599670,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599677,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5072,"Index":62,"Attempt":0,"Launch Time":1427397599664,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599670,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":331337,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5078,"Index":68,"Attempt":0,"Launch Time":1427397599673,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599686,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5071,"Index":61,"Attempt":0,"Launch Time":1427397599661,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599674,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":327468,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5079,"Index":69,"Attempt":0,"Launch Time":1427397599675,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599684,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5074,"Index":64,"Attempt":0,"Launch Time":1427397599667,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599675,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":340004,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5080,"Index":70,"Attempt":0,"Launch Time":1427397599677,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599695,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5077,"Index":67,"Attempt":0,"Launch Time":1427397599670,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599677,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9614,"Shuffle Write Time":259332,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5081,"Index":71,"Attempt":0,"Launch Time":1427397599679,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599696,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5075,"Index":65,"Attempt":0,"Launch Time":1427397599668,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599679,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":301673,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5082,"Index":72,"Attempt":0,"Launch Time":1427397599681,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599695,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5073,"Index":63,"Attempt":0,"Launch Time":1427397599666,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599681,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":363926,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5083,"Index":73,"Attempt":0,"Launch Time":1427397599683,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599690,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5079,"Index":69,"Attempt":0,"Launch Time":1427397599675,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599684,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":488890,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5084,"Index":74,"Attempt":0,"Launch Time":1427397599685,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599696,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5078,"Index":68,"Attempt":0,"Launch Time":1427397599673,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599686,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":303759,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5085,"Index":75,"Attempt":0,"Launch Time":1427397599687,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599694,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5076,"Index":66,"Attempt":0,"Launch Time":1427397599669,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599687,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1052247,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5086,"Index":76,"Attempt":0,"Launch Time":1427397599690,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5083,"Index":73,"Attempt":0,"Launch Time":1427397599683,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599690,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":341467,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5087,"Index":77,"Attempt":0,"Launch Time":1427397599690,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599697,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5058,"Index":48,"Attempt":0,"Launch Time":1427397599640,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599691,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":48,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":10442141,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5088,"Index":78,"Attempt":0,"Launch Time":1427397599694,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599704,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5085,"Index":75,"Attempt":0,"Launch Time":1427397599687,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599694,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":326266,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5089,"Index":79,"Attempt":0,"Launch Time":1427397599695,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599706,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5082,"Index":72,"Attempt":0,"Launch Time":1427397599681,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599695,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":336528,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5090,"Index":80,"Attempt":0,"Launch Time":1427397599695,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599703,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5080,"Index":70,"Attempt":0,"Launch Time":1427397599677,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599695,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":704043,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5091,"Index":81,"Attempt":0,"Launch Time":1427397599696,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599704,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5084,"Index":74,"Attempt":0,"Launch Time":1427397599685,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599696,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":406415,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5092,"Index":82,"Attempt":0,"Launch Time":1427397599696,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599707,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5081,"Index":71,"Attempt":0,"Launch Time":1427397599679,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599696,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":440434,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5093,"Index":83,"Attempt":0,"Launch Time":1427397599697,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599705,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5087,"Index":77,"Attempt":0,"Launch Time":1427397599690,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599697,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":321902,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5094,"Index":84,"Attempt":0,"Launch Time":1427397599703,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5090,"Index":80,"Attempt":0,"Launch Time":1427397599695,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599703,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":319125,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5095,"Index":85,"Attempt":0,"Launch Time":1427397599704,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5091,"Index":81,"Attempt":0,"Launch Time":1427397599696,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599704,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":314890,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5096,"Index":86,"Attempt":0,"Launch Time":1427397599704,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5088,"Index":78,"Attempt":0,"Launch Time":1427397599694,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599704,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":446729,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5097,"Index":87,"Attempt":0,"Launch Time":1427397599704,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599718,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5093,"Index":83,"Attempt":0,"Launch Time":1427397599697,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599705,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":428985,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5098,"Index":88,"Attempt":0,"Launch Time":1427397599706,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5089,"Index":79,"Attempt":0,"Launch Time":1427397599695,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599706,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":604707,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5099,"Index":89,"Attempt":0,"Launch Time":1427397599706,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5092,"Index":82,"Attempt":0,"Launch Time":1427397599696,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599707,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":382178,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5100,"Index":90,"Attempt":0,"Launch Time":1427397599718,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5097,"Index":87,"Attempt":0,"Launch Time":1427397599704,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599718,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":303917,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5101,"Index":91,"Attempt":0,"Launch Time":1427397599724,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5096,"Index":86,"Attempt":0,"Launch Time":1427397599704,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599724,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":321469,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5102,"Index":92,"Attempt":0,"Launch Time":1427397599727,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5054,"Index":44,"Attempt":0,"Launch Time":1427397599616,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599727,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":110,"Result Size":930,"JVM GC Time":22,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":299315,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5103,"Index":93,"Attempt":0,"Launch Time":1427397599729,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5100,"Index":90,"Attempt":0,"Launch Time":1427397599718,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599729,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":307060,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5104,"Index":94,"Attempt":0,"Launch Time":1427397599733,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5098,"Index":88,"Attempt":0,"Launch Time":1427397599706,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599733,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":26,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":345687,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5105,"Index":95,"Attempt":0,"Launch Time":1427397599733,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5099,"Index":89,"Attempt":0,"Launch Time":1427397599706,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599733,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":26,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":517460,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5106,"Index":96,"Attempt":0,"Launch Time":1427397599735,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5094,"Index":84,"Attempt":0,"Launch Time":1427397599703,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599735,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":31,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":332600,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5107,"Index":97,"Attempt":0,"Launch Time":1427397599736,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5102,"Index":92,"Attempt":0,"Launch Time":1427397599727,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599736,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":387545,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5108,"Index":98,"Attempt":0,"Launch Time":1427397599736,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5103,"Index":93,"Attempt":0,"Launch Time":1427397599729,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599736,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":318561,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5109,"Index":99,"Attempt":0,"Launch Time":1427397599738,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5101,"Index":91,"Attempt":0,"Launch Time":1427397599724,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599738,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1663815,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5110,"Index":100,"Attempt":0,"Launch Time":1427397599739,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5095,"Index":85,"Attempt":0,"Launch Time":1427397599704,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599739,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":35,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":574348,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5111,"Index":101,"Attempt":0,"Launch Time":1427397599740,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5104,"Index":94,"Attempt":0,"Launch Time":1427397599733,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599740,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":327802,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5112,"Index":102,"Attempt":0,"Launch Time":1427397599743,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5107,"Index":97,"Attempt":0,"Launch Time":1427397599736,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599743,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":333263,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5113,"Index":103,"Attempt":0,"Launch Time":1427397599744,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5108,"Index":98,"Attempt":0,"Launch Time":1427397599736,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599744,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":361129,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5114,"Index":104,"Attempt":0,"Launch Time":1427397599746,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5106,"Index":96,"Attempt":0,"Launch Time":1427397599735,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599746,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":322313,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5115,"Index":105,"Attempt":0,"Launch Time":1427397599746,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5105,"Index":95,"Attempt":0,"Launch Time":1427397599733,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599747,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":711638,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5116,"Index":106,"Attempt":0,"Launch Time":1427397599747,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5109,"Index":99,"Attempt":0,"Launch Time":1427397599738,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599747,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":346158,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5117,"Index":107,"Attempt":0,"Launch Time":1427397599749,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5111,"Index":101,"Attempt":0,"Launch Time":1427397599740,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599750,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":492381,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5118,"Index":108,"Attempt":0,"Launch Time":1427397599752,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5113,"Index":103,"Attempt":0,"Launch Time":1427397599744,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599752,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":457505,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5119,"Index":109,"Attempt":0,"Launch Time":1427397599753,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5112,"Index":102,"Attempt":0,"Launch Time":1427397599743,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599753,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":581901,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5120,"Index":110,"Attempt":0,"Launch Time":1427397599753,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5110,"Index":100,"Attempt":0,"Launch Time":1427397599739,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599753,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":848349,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5121,"Index":111,"Attempt":0,"Launch Time":1427397599755,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5114,"Index":104,"Attempt":0,"Launch Time":1427397599746,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599755,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":357847,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5122,"Index":112,"Attempt":0,"Launch Time":1427397599756,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5123,"Index":113,"Attempt":0,"Launch Time":1427397599757,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5117,"Index":107,"Attempt":0,"Launch Time":1427397599749,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599757,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":409267,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5115,"Index":105,"Attempt":0,"Launch Time":1427397599746,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599757,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":349418,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5124,"Index":114,"Attempt":0,"Launch Time":1427397599759,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5116,"Index":106,"Attempt":0,"Launch Time":1427397599747,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599759,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":320162,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5125,"Index":115,"Attempt":0,"Launch Time":1427397599762,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5120,"Index":110,"Attempt":0,"Launch Time":1427397599753,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599762,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":318743,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5126,"Index":116,"Attempt":0,"Launch Time":1427397599763,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5119,"Index":109,"Attempt":0,"Launch Time":1427397599753,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599763,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":301181,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5127,"Index":117,"Attempt":0,"Launch Time":1427397599769,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5118,"Index":108,"Attempt":0,"Launch Time":1427397599752,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599769,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":338343,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5128,"Index":118,"Attempt":0,"Launch Time":1427397599770,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5125,"Index":115,"Attempt":0,"Launch Time":1427397599762,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599770,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":505917,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5129,"Index":119,"Attempt":0,"Launch Time":1427397599771,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5086,"Index":76,"Attempt":0,"Launch Time":1427397599690,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599771,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":5,"Executor Run Time":75,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":332074,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5130,"Index":120,"Attempt":0,"Launch Time":1427397599771,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5121,"Index":111,"Attempt":0,"Launch Time":1427397599755,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599772,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":291620,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5131,"Index":121,"Attempt":0,"Launch Time":1427397599774,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5126,"Index":116,"Attempt":0,"Launch Time":1427397599763,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599775,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":575200,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5132,"Index":122,"Attempt":0,"Launch Time":1427397599775,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5127,"Index":117,"Attempt":0,"Launch Time":1427397599769,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599775,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":286242,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5133,"Index":123,"Attempt":0,"Launch Time":1427397599777,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5129,"Index":119,"Attempt":0,"Launch Time":1427397599771,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599777,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":299613,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5134,"Index":124,"Attempt":0,"Launch Time":1427397599782,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5133,"Index":123,"Attempt":0,"Launch Time":1427397599777,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599783,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":290953,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5135,"Index":125,"Attempt":0,"Launch Time":1427397599786,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5131,"Index":121,"Attempt":0,"Launch Time":1427397599774,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599786,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":329705,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5136,"Index":126,"Attempt":0,"Launch Time":1427397599788,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5134,"Index":124,"Attempt":0,"Launch Time":1427397599782,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599789,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":279741,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5137,"Index":127,"Attempt":0,"Launch Time":1427397599792,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5130,"Index":120,"Attempt":0,"Launch Time":1427397599771,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599792,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":302567,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5138,"Index":128,"Attempt":0,"Launch Time":1427397599793,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5135,"Index":125,"Attempt":0,"Launch Time":1427397599786,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599793,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":316676,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5139,"Index":129,"Attempt":0,"Launch Time":1427397599794,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5132,"Index":122,"Attempt":0,"Launch Time":1427397599775,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599794,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":303257,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5140,"Index":130,"Attempt":0,"Launch Time":1427397599795,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5136,"Index":126,"Attempt":0,"Launch Time":1427397599788,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599795,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":290375,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5141,"Index":131,"Attempt":0,"Launch Time":1427397599800,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5138,"Index":128,"Attempt":0,"Launch Time":1427397599793,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599801,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":315981,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5142,"Index":132,"Attempt":0,"Launch Time":1427397599803,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5140,"Index":130,"Attempt":0,"Launch Time":1427397599795,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599803,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":332673,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5143,"Index":133,"Attempt":0,"Launch Time":1427397599803,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5137,"Index":127,"Attempt":0,"Launch Time":1427397599792,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599804,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":341526,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5144,"Index":134,"Attempt":0,"Launch Time":1427397599805,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5139,"Index":129,"Attempt":0,"Launch Time":1427397599794,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599806,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":348599,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5145,"Index":135,"Attempt":0,"Launch Time":1427397599813,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5122,"Index":112,"Attempt":0,"Launch Time":1427397599756,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599813,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":47,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":309994,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5146,"Index":136,"Attempt":0,"Launch Time":1427397599815,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5142,"Index":132,"Attempt":0,"Launch Time":1427397599803,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599815,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":327826,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5147,"Index":137,"Attempt":0,"Launch Time":1427397599816,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5143,"Index":133,"Attempt":0,"Launch Time":1427397599803,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599816,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":323115,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5148,"Index":138,"Attempt":0,"Launch Time":1427397599817,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5123,"Index":113,"Attempt":0,"Launch Time":1427397599757,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599817,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":56,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":770436,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5149,"Index":139,"Attempt":0,"Launch Time":1427397599822,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599828,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5145,"Index":135,"Attempt":0,"Launch Time":1427397599813,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599822,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":416045,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5150,"Index":140,"Attempt":0,"Launch Time":1427397599822,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599828,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5146,"Index":136,"Attempt":0,"Launch Time":1427397599815,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599822,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":335994,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5151,"Index":141,"Attempt":0,"Launch Time":1427397599823,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599836,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5147,"Index":137,"Attempt":0,"Launch Time":1427397599816,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599824,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":342243,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5152,"Index":142,"Attempt":0,"Launch Time":1427397599828,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599836,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5149,"Index":139,"Attempt":0,"Launch Time":1427397599822,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599828,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":284553,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5153,"Index":143,"Attempt":0,"Launch Time":1427397599828,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599836,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5150,"Index":140,"Attempt":0,"Launch Time":1427397599822,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599828,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9606,"Shuffle Write Time":289035,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5154,"Index":144,"Attempt":0,"Launch Time":1427397599835,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5155,"Index":145,"Attempt":0,"Launch Time":1427397599836,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5156,"Index":146,"Attempt":0,"Launch Time":1427397599836,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5141,"Index":131,"Attempt":0,"Launch Time":1427397599800,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599836,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":29,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":22680854,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5148,"Index":138,"Attempt":0,"Launch Time":1427397599817,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599836,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":5,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":297531,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5157,"Index":147,"Attempt":0,"Launch Time":1427397599836,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5158,"Index":148,"Attempt":0,"Launch Time":1427397599836,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5152,"Index":142,"Attempt":0,"Launch Time":1427397599828,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599836,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":326003,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5151,"Index":141,"Attempt":0,"Launch Time":1427397599823,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599836,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":283701,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5159,"Index":149,"Attempt":0,"Launch Time":1427397599836,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5153,"Index":143,"Attempt":0,"Launch Time":1427397599828,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599836,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":319799,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5124,"Index":114,"Attempt":0,"Launch Time":1427397599759,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599837,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":67,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":332265,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5160,"Index":150,"Attempt":0,"Launch Time":1427397599837,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5144,"Index":134,"Attempt":0,"Launch Time":1427397599805,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599837,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":29,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":484934,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5161,"Index":151,"Attempt":0,"Launch Time":1427397599844,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5154,"Index":144,"Attempt":0,"Launch Time":1427397599835,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599844,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":334357,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5162,"Index":152,"Attempt":0,"Launch Time":1427397599844,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5159,"Index":149,"Attempt":0,"Launch Time":1427397599836,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599844,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":334249,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5163,"Index":153,"Attempt":0,"Launch Time":1427397599845,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5158,"Index":148,"Attempt":0,"Launch Time":1427397599836,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599845,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":346497,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5164,"Index":154,"Attempt":0,"Launch Time":1427397599846,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5155,"Index":145,"Attempt":0,"Launch Time":1427397599836,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599846,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":323539,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5165,"Index":155,"Attempt":0,"Launch Time":1427397599852,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5163,"Index":153,"Attempt":0,"Launch Time":1427397599845,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599852,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":356783,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5166,"Index":156,"Attempt":0,"Launch Time":1427397599853,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5161,"Index":151,"Attempt":0,"Launch Time":1427397599844,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599853,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":386707,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5167,"Index":157,"Attempt":0,"Launch Time":1427397599853,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5162,"Index":152,"Attempt":0,"Launch Time":1427397599844,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599853,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":391681,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5168,"Index":158,"Attempt":0,"Launch Time":1427397599856,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5169,"Index":159,"Attempt":0,"Launch Time":1427397599856,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5164,"Index":154,"Attempt":0,"Launch Time":1427397599846,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599856,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":374866,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5156,"Index":146,"Attempt":0,"Launch Time":1427397599836,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599856,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":494644,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5170,"Index":160,"Attempt":0,"Launch Time":1427397599858,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5160,"Index":150,"Attempt":0,"Launch Time":1427397599837,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599858,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":350706,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5171,"Index":161,"Attempt":0,"Launch Time":1427397599860,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5166,"Index":156,"Attempt":0,"Launch Time":1427397599853,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599860,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":387185,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5172,"Index":162,"Attempt":0,"Launch Time":1427397599860,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5167,"Index":157,"Attempt":0,"Launch Time":1427397599853,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599860,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":344981,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5173,"Index":163,"Attempt":0,"Launch Time":1427397599861,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5165,"Index":155,"Attempt":0,"Launch Time":1427397599852,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599861,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":442425,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5174,"Index":164,"Attempt":0,"Launch Time":1427397599862,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5168,"Index":158,"Attempt":0,"Launch Time":1427397599856,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599862,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":285468,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5175,"Index":165,"Attempt":0,"Launch Time":1427397599864,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5157,"Index":147,"Attempt":0,"Launch Time":1427397599836,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599864,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":326685,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5176,"Index":166,"Attempt":0,"Launch Time":1427397599865,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5169,"Index":159,"Attempt":0,"Launch Time":1427397599856,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599865,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":300784,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5177,"Index":167,"Attempt":0,"Launch Time":1427397599866,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5172,"Index":162,"Attempt":0,"Launch Time":1427397599860,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599866,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":293617,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5178,"Index":168,"Attempt":0,"Launch Time":1427397599866,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5179,"Index":169,"Attempt":0,"Launch Time":1427397599866,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5171,"Index":161,"Attempt":0,"Launch Time":1427397599860,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599867,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":276797,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5173,"Index":163,"Attempt":0,"Launch Time":1427397599861,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599867,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":267617,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5180,"Index":170,"Attempt":0,"Launch Time":1427397599868,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5174,"Index":164,"Attempt":0,"Launch Time":1427397599862,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599868,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":290102,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5181,"Index":171,"Attempt":0,"Launch Time":1427397599869,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5175,"Index":165,"Attempt":0,"Launch Time":1427397599864,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599870,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":257298,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5182,"Index":172,"Attempt":0,"Launch Time":1427397599873,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5178,"Index":168,"Attempt":0,"Launch Time":1427397599866,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599873,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":347702,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5183,"Index":173,"Attempt":0,"Launch Time":1427397599874,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5177,"Index":167,"Attempt":0,"Launch Time":1427397599866,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599874,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":384686,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5184,"Index":174,"Attempt":0,"Launch Time":1427397599876,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5176,"Index":166,"Attempt":0,"Launch Time":1427397599865,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599876,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1597738,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5185,"Index":175,"Attempt":0,"Launch Time":1427397599880,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5182,"Index":172,"Attempt":0,"Launch Time":1427397599873,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599880,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":344960,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5186,"Index":176,"Attempt":0,"Launch Time":1427397599880,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5183,"Index":173,"Attempt":0,"Launch Time":1427397599874,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599880,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":272591,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5187,"Index":177,"Attempt":0,"Launch Time":1427397599881,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5181,"Index":171,"Attempt":0,"Launch Time":1427397599869,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599881,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":248233,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5188,"Index":178,"Attempt":0,"Launch Time":1427397599883,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5189,"Index":179,"Attempt":0,"Launch Time":1427397599883,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5184,"Index":174,"Attempt":0,"Launch Time":1427397599876,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599883,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":258902,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5170,"Index":160,"Attempt":0,"Launch Time":1427397599858,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599883,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":21,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1337090,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5190,"Index":180,"Attempt":0,"Launch Time":1427397599883,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5180,"Index":170,"Attempt":0,"Launch Time":1427397599868,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599883,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":328975,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5191,"Index":181,"Attempt":0,"Launch Time":1427397599885,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5185,"Index":175,"Attempt":0,"Launch Time":1427397599880,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599886,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":285928,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5192,"Index":182,"Attempt":0,"Launch Time":1427397599889,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5189,"Index":179,"Attempt":0,"Launch Time":1427397599883,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599890,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":337649,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5193,"Index":183,"Attempt":0,"Launch Time":1427397599890,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5187,"Index":177,"Attempt":0,"Launch Time":1427397599881,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599890,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":352526,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5194,"Index":184,"Attempt":0,"Launch Time":1427397599903,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5128,"Index":118,"Attempt":0,"Launch Time":1427397599770,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599903,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":32,"Executor Run Time":58,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":405099,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5195,"Index":185,"Attempt":0,"Launch Time":1427397599904,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5190,"Index":180,"Attempt":0,"Launch Time":1427397599883,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599904,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":365683,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5196,"Index":186,"Attempt":0,"Launch Time":1427397599905,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5193,"Index":183,"Attempt":0,"Launch Time":1427397599890,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599906,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":342432,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5197,"Index":187,"Attempt":0,"Launch Time":1427397599910,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5194,"Index":184,"Attempt":0,"Launch Time":1427397599903,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599910,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":286315,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5198,"Index":188,"Attempt":0,"Launch Time":1427397599912,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5195,"Index":185,"Attempt":0,"Launch Time":1427397599904,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599912,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":318573,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5199,"Index":189,"Attempt":0,"Launch Time":1427397599914,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5196,"Index":186,"Attempt":0,"Launch Time":1427397599905,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599914,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":342782,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5200,"Index":190,"Attempt":0,"Launch Time":1427397599916,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5197,"Index":187,"Attempt":0,"Launch Time":1427397599910,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599916,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":262527,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5201,"Index":191,"Attempt":0,"Launch Time":1427397599919,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5198,"Index":188,"Attempt":0,"Launch Time":1427397599912,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599919,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":296255,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5202,"Index":192,"Attempt":0,"Launch Time":1427397599919,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5188,"Index":178,"Attempt":0,"Launch Time":1427397599883,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599920,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":36,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":437603,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5203,"Index":193,"Attempt":0,"Launch Time":1427397599921,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5199,"Index":189,"Attempt":0,"Launch Time":1427397599914,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599921,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":307266,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5204,"Index":194,"Attempt":0,"Launch Time":1427397599924,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5200,"Index":190,"Attempt":0,"Launch Time":1427397599916,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599924,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":307790,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5205,"Index":195,"Attempt":0,"Launch Time":1427397599925,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5201,"Index":191,"Attempt":0,"Launch Time":1427397599919,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599925,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":520614,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5206,"Index":196,"Attempt":0,"Launch Time":1427397599928,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5202,"Index":192,"Attempt":0,"Launch Time":1427397599919,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599928,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":306660,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5207,"Index":197,"Attempt":0,"Launch Time":1427397599929,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5203,"Index":193,"Attempt":0,"Launch Time":1427397599921,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599929,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":291512,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5208,"Index":198,"Attempt":0,"Launch Time":1427397599931,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5204,"Index":194,"Attempt":0,"Launch Time":1427397599924,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599931,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":290751,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5209,"Index":199,"Attempt":0,"Launch Time":1427397599931,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5205,"Index":195,"Attempt":0,"Launch Time":1427397599925,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599932,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":291081,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5210,"Index":200,"Attempt":0,"Launch Time":1427397599935,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5207,"Index":197,"Attempt":0,"Launch Time":1427397599929,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599936,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":479914,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5211,"Index":201,"Attempt":0,"Launch Time":1427397599937,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5208,"Index":198,"Attempt":0,"Launch Time":1427397599931,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599937,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":289331,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5212,"Index":202,"Attempt":0,"Launch Time":1427397599941,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5209,"Index":199,"Attempt":0,"Launch Time":1427397599931,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599941,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":319155,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5213,"Index":203,"Attempt":0,"Launch Time":1427397599943,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5210,"Index":200,"Attempt":0,"Launch Time":1427397599935,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599943,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":304125,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5214,"Index":204,"Attempt":0,"Launch Time":1427397599944,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5211,"Index":201,"Attempt":0,"Launch Time":1427397599937,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599944,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":316345,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5215,"Index":205,"Attempt":0,"Launch Time":1427397599947,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5212,"Index":202,"Attempt":0,"Launch Time":1427397599941,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599947,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":363297,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5216,"Index":206,"Attempt":0,"Launch Time":1427397599950,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5213,"Index":203,"Attempt":0,"Launch Time":1427397599943,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599950,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":339923,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5217,"Index":207,"Attempt":0,"Launch Time":1427397599951,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5214,"Index":204,"Attempt":0,"Launch Time":1427397599944,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599951,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":367487,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5218,"Index":208,"Attempt":0,"Launch Time":1427397599955,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5215,"Index":205,"Attempt":0,"Launch Time":1427397599947,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599955,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":365146,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5219,"Index":209,"Attempt":0,"Launch Time":1427397599958,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5216,"Index":206,"Attempt":0,"Launch Time":1427397599950,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599958,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":310317,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5220,"Index":210,"Attempt":0,"Launch Time":1427397599958,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5217,"Index":207,"Attempt":0,"Launch Time":1427397599951,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599958,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":356003,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5221,"Index":211,"Attempt":0,"Launch Time":1427397599963,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5218,"Index":208,"Attempt":0,"Launch Time":1427397599955,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599963,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":280371,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5222,"Index":212,"Attempt":0,"Launch Time":1427397599964,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5219,"Index":209,"Attempt":0,"Launch Time":1427397599958,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599965,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":323104,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5223,"Index":213,"Attempt":0,"Launch Time":1427397599965,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5220,"Index":210,"Attempt":0,"Launch Time":1427397599958,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599965,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":293165,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5224,"Index":214,"Attempt":0,"Launch Time":1427397599968,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5192,"Index":182,"Attempt":0,"Launch Time":1427397599889,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599968,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":78,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":321195,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5225,"Index":215,"Attempt":0,"Launch Time":1427397599971,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5221,"Index":211,"Attempt":0,"Launch Time":1427397599963,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599971,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":369873,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5226,"Index":216,"Attempt":0,"Launch Time":1427397599972,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5223,"Index":213,"Attempt":0,"Launch Time":1427397599965,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599972,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":308723,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5227,"Index":217,"Attempt":0,"Launch Time":1427397599972,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5222,"Index":212,"Attempt":0,"Launch Time":1427397599964,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599972,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":398784,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5228,"Index":218,"Attempt":0,"Launch Time":1427397599974,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5224,"Index":214,"Attempt":0,"Launch Time":1427397599968,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599974,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":327612,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5229,"Index":219,"Attempt":0,"Launch Time":1427397599976,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5191,"Index":181,"Attempt":0,"Launch Time":1427397599885,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599976,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":90,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":344199,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5230,"Index":220,"Attempt":0,"Launch Time":1427397599981,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5179,"Index":169,"Attempt":0,"Launch Time":1427397599866,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599981,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":114,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":276876,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5231,"Index":221,"Attempt":0,"Launch Time":1427397599983,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5226,"Index":216,"Attempt":0,"Launch Time":1427397599972,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599983,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":290879,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5232,"Index":222,"Attempt":0,"Launch Time":1427397599985,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5225,"Index":215,"Attempt":0,"Launch Time":1427397599971,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599986,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":317210,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5233,"Index":223,"Attempt":0,"Launch Time":1427397599986,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5227,"Index":217,"Attempt":0,"Launch Time":1427397599972,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599986,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":299488,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5234,"Index":224,"Attempt":0,"Launch Time":1427397599989,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5206,"Index":196,"Attempt":0,"Launch Time":1427397599928,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599989,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":57,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":314467,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5235,"Index":225,"Attempt":0,"Launch Time":1427397599990,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5231,"Index":221,"Attempt":0,"Launch Time":1427397599983,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599990,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":311192,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5236,"Index":226,"Attempt":0,"Launch Time":1427397600003,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5232,"Index":222,"Attempt":0,"Launch Time":1427397599985,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600003,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":623303,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5237,"Index":227,"Attempt":0,"Launch Time":1427397600006,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5233,"Index":223,"Attempt":0,"Launch Time":1427397599986,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600006,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":20,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1216413,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5238,"Index":228,"Attempt":0,"Launch Time":1427397600010,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5235,"Index":225,"Attempt":0,"Launch Time":1427397599990,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600010,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":18,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":327138,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5239,"Index":229,"Attempt":0,"Launch Time":1427397600011,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5234,"Index":224,"Attempt":0,"Launch Time":1427397599989,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600011,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":22,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":4753634,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5240,"Index":230,"Attempt":0,"Launch Time":1427397600012,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5236,"Index":226,"Attempt":0,"Launch Time":1427397600003,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600012,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":285032,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5241,"Index":231,"Attempt":0,"Launch Time":1427397600014,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5237,"Index":227,"Attempt":0,"Launch Time":1427397600006,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600014,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":306783,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5242,"Index":232,"Attempt":0,"Launch Time":1427397600016,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5230,"Index":220,"Attempt":0,"Launch Time":1427397599981,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600017,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":31,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":452882,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5243,"Index":233,"Attempt":0,"Launch Time":1427397600018,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5238,"Index":228,"Attempt":0,"Launch Time":1427397600010,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600018,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":456877,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5239,"Index":229,"Attempt":0,"Launch Time":1427397600011,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600020,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":379190,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5244,"Index":234,"Attempt":0,"Launch Time":1427397600020,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5245,"Index":235,"Attempt":0,"Launch Time":1427397600021,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5240,"Index":230,"Attempt":0,"Launch Time":1427397600012,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600021,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":281622,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5246,"Index":236,"Attempt":0,"Launch Time":1427397600022,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5241,"Index":231,"Attempt":0,"Launch Time":1427397600014,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600022,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":310997,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5247,"Index":237,"Attempt":0,"Launch Time":1427397600023,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5242,"Index":232,"Attempt":0,"Launch Time":1427397600016,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600023,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":301727,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5248,"Index":238,"Attempt":0,"Launch Time":1427397600027,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5249,"Index":239,"Attempt":0,"Launch Time":1427397600027,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5186,"Index":176,"Attempt":0,"Launch Time":1427397599880,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600027,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":147,"Result Size":930,"JVM GC Time":12,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":317390,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5229,"Index":219,"Attempt":0,"Launch Time":1427397599976,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600027,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":19,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":258967,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5250,"Index":240,"Attempt":0,"Launch Time":1427397600028,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5228,"Index":218,"Attempt":0,"Launch Time":1427397599974,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600029,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":53,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":6449843,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5251,"Index":241,"Attempt":0,"Launch Time":1427397600033,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5247,"Index":237,"Attempt":0,"Launch Time":1427397600023,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600033,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":333224,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5252,"Index":242,"Attempt":0,"Launch Time":1427397600034,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5248,"Index":238,"Attempt":0,"Launch Time":1427397600027,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600034,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":340833,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5253,"Index":243,"Attempt":0,"Launch Time":1427397600034,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5249,"Index":239,"Attempt":0,"Launch Time":1427397600027,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600034,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":301221,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5254,"Index":244,"Attempt":0,"Launch Time":1427397600039,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5250,"Index":240,"Attempt":0,"Launch Time":1427397600028,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600039,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":297767,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5255,"Index":245,"Attempt":0,"Launch Time":1427397600042,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5245,"Index":235,"Attempt":0,"Launch Time":1427397600021,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600042,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":337330,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5256,"Index":246,"Attempt":0,"Launch Time":1427397600044,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5253,"Index":243,"Attempt":0,"Launch Time":1427397600034,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600044,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":323534,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5257,"Index":247,"Attempt":0,"Launch Time":1427397600044,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5244,"Index":234,"Attempt":0,"Launch Time":1427397600020,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600044,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1397633,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5258,"Index":248,"Attempt":0,"Launch Time":1427397600045,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5252,"Index":242,"Attempt":0,"Launch Time":1427397600034,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600045,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":323449,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5259,"Index":249,"Attempt":0,"Launch Time":1427397600047,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5251,"Index":241,"Attempt":0,"Launch Time":1427397600033,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600047,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":310857,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5260,"Index":250,"Attempt":0,"Launch Time":1427397600048,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5254,"Index":244,"Attempt":0,"Launch Time":1427397600039,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600049,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":368038,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5261,"Index":251,"Attempt":0,"Launch Time":1427397600051,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5257,"Index":247,"Attempt":0,"Launch Time":1427397600044,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600052,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":367849,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5262,"Index":252,"Attempt":0,"Launch Time":1427397600052,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5256,"Index":246,"Attempt":0,"Launch Time":1427397600044,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600053,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":361785,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5263,"Index":253,"Attempt":0,"Launch Time":1427397600056,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5258,"Index":248,"Attempt":0,"Launch Time":1427397600045,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600056,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":523419,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5264,"Index":254,"Attempt":0,"Launch Time":1427397600058,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5259,"Index":249,"Attempt":0,"Launch Time":1427397600047,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600058,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9613,"Shuffle Write Time":593172,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5265,"Index":255,"Attempt":0,"Launch Time":1427397600058,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5260,"Index":250,"Attempt":0,"Launch Time":1427397600048,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600058,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":353479,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5266,"Index":256,"Attempt":0,"Launch Time":1427397600060,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5261,"Index":251,"Attempt":0,"Launch Time":1427397600051,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600060,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":446753,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5267,"Index":257,"Attempt":0,"Launch Time":1427397600064,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5263,"Index":253,"Attempt":0,"Launch Time":1427397600056,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600064,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":362429,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5268,"Index":258,"Attempt":0,"Launch Time":1427397600064,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5262,"Index":252,"Attempt":0,"Launch Time":1427397600052,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600064,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":308408,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5269,"Index":259,"Attempt":0,"Launch Time":1427397600065,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5265,"Index":255,"Attempt":0,"Launch Time":1427397600058,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600065,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":299826,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5270,"Index":260,"Attempt":0,"Launch Time":1427397600066,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5266,"Index":256,"Attempt":0,"Launch Time":1427397600060,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600067,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":289866,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5271,"Index":261,"Attempt":0,"Launch Time":1427397600070,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5267,"Index":257,"Attempt":0,"Launch Time":1427397600064,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600070,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9613,"Shuffle Write Time":277152,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5272,"Index":262,"Attempt":0,"Launch Time":1427397600070,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5268,"Index":258,"Attempt":0,"Launch Time":1427397600064,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600071,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":301405,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5273,"Index":263,"Attempt":0,"Launch Time":1427397600071,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5269,"Index":259,"Attempt":0,"Launch Time":1427397600065,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600071,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":346844,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5274,"Index":264,"Attempt":0,"Launch Time":1427397600075,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5270,"Index":260,"Attempt":0,"Launch Time":1427397600066,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600075,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":507880,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5275,"Index":265,"Attempt":0,"Launch Time":1427397600076,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5271,"Index":261,"Attempt":0,"Launch Time":1427397600070,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600076,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":260991,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5276,"Index":266,"Attempt":0,"Launch Time":1427397600077,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5273,"Index":263,"Attempt":0,"Launch Time":1427397600071,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600077,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":297896,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5277,"Index":267,"Attempt":0,"Launch Time":1427397600078,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5272,"Index":262,"Attempt":0,"Launch Time":1427397600070,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600078,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":301116,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5278,"Index":268,"Attempt":0,"Launch Time":1427397600078,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5264,"Index":254,"Attempt":0,"Launch Time":1427397600058,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600078,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":19,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":8980526,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5279,"Index":269,"Attempt":0,"Launch Time":1427397600081,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5274,"Index":264,"Attempt":0,"Launch Time":1427397600075,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600081,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":287842,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5280,"Index":270,"Attempt":0,"Launch Time":1427397600083,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5275,"Index":265,"Attempt":0,"Launch Time":1427397600076,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600083,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9613,"Shuffle Write Time":304906,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5281,"Index":271,"Attempt":0,"Launch Time":1427397600084,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5276,"Index":266,"Attempt":0,"Launch Time":1427397600077,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600096,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":315414,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5282,"Index":272,"Attempt":0,"Launch Time":1427397600096,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5283,"Index":273,"Attempt":0,"Launch Time":1427397600096,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5246,"Index":236,"Attempt":0,"Launch Time":1427397600022,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600096,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":311151,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5279,"Index":269,"Attempt":0,"Launch Time":1427397600081,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600096,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":284270,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5284,"Index":274,"Attempt":0,"Launch Time":1427397600096,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5280,"Index":270,"Attempt":0,"Launch Time":1427397600083,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600096,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":312412,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5285,"Index":275,"Attempt":0,"Launch Time":1427397600099,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5278,"Index":268,"Attempt":0,"Launch Time":1427397600078,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600099,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":20,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":11663284,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5286,"Index":276,"Attempt":0,"Launch Time":1427397600101,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5277,"Index":267,"Attempt":0,"Launch Time":1427397600078,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600102,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":22,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":345466,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5287,"Index":277,"Attempt":0,"Launch Time":1427397600102,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5281,"Index":271,"Attempt":0,"Launch Time":1427397600084,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600102,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":344209,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5288,"Index":278,"Attempt":0,"Launch Time":1427397600103,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5283,"Index":273,"Attempt":0,"Launch Time":1427397600096,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600103,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":335820,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5289,"Index":279,"Attempt":0,"Launch Time":1427397600105,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5284,"Index":274,"Attempt":0,"Launch Time":1427397600096,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600105,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":394971,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5290,"Index":280,"Attempt":0,"Launch Time":1427397600107,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5285,"Index":275,"Attempt":0,"Launch Time":1427397600099,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600108,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":350171,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5291,"Index":281,"Attempt":0,"Launch Time":1427397600109,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5286,"Index":276,"Attempt":0,"Launch Time":1427397600101,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600110,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":343340,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5292,"Index":282,"Attempt":0,"Launch Time":1427397600110,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5288,"Index":278,"Attempt":0,"Launch Time":1427397600103,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600110,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":287674,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5293,"Index":283,"Attempt":0,"Launch Time":1427397600111,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5287,"Index":277,"Attempt":0,"Launch Time":1427397600102,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600112,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":319817,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5294,"Index":284,"Attempt":0,"Launch Time":1427397600117,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5292,"Index":282,"Attempt":0,"Launch Time":1427397600110,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600117,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":293170,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5295,"Index":285,"Attempt":0,"Launch Time":1427397600120,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5290,"Index":280,"Attempt":0,"Launch Time":1427397600107,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600120,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":328056,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5296,"Index":286,"Attempt":0,"Launch Time":1427397600121,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5291,"Index":281,"Attempt":0,"Launch Time":1427397600109,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600122,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":334607,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5297,"Index":287,"Attempt":0,"Launch Time":1427397600123,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5294,"Index":284,"Attempt":0,"Launch Time":1427397600117,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600124,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":318380,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5298,"Index":288,"Attempt":0,"Launch Time":1427397600124,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5289,"Index":279,"Attempt":0,"Launch Time":1427397600105,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600124,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":310163,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5299,"Index":289,"Attempt":0,"Launch Time":1427397600128,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5300,"Index":290,"Attempt":0,"Launch Time":1427397600128,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5295,"Index":285,"Attempt":0,"Launch Time":1427397600120,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600129,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":331174,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5296,"Index":286,"Attempt":0,"Launch Time":1427397600121,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600129,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":307354,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5301,"Index":291,"Attempt":0,"Launch Time":1427397600130,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5297,"Index":287,"Attempt":0,"Launch Time":1427397600123,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600130,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":292616,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5302,"Index":292,"Attempt":0,"Launch Time":1427397600134,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5255,"Index":245,"Attempt":0,"Launch Time":1427397600042,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600134,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":14,"Executor Run Time":30,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":334466,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5303,"Index":293,"Attempt":0,"Launch Time":1427397600135,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5298,"Index":288,"Attempt":0,"Launch Time":1427397600124,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600135,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":321344,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5304,"Index":294,"Attempt":0,"Launch Time":1427397600136,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5300,"Index":290,"Attempt":0,"Launch Time":1427397600128,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600136,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":348925,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5305,"Index":295,"Attempt":0,"Launch Time":1427397600136,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5301,"Index":291,"Attempt":0,"Launch Time":1427397600130,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600136,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":321799,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5306,"Index":296,"Attempt":0,"Launch Time":1427397600141,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5293,"Index":283,"Attempt":0,"Launch Time":1427397600111,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600141,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":28,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":355778,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5307,"Index":297,"Attempt":0,"Launch Time":1427397600142,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5304,"Index":294,"Attempt":0,"Launch Time":1427397600136,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600143,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":311896,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5308,"Index":298,"Attempt":0,"Launch Time":1427397600143,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5305,"Index":295,"Attempt":0,"Launch Time":1427397600136,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600143,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":293874,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5309,"Index":299,"Attempt":0,"Launch Time":1427397600143,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5303,"Index":293,"Attempt":0,"Launch Time":1427397600135,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600143,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":313339,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5310,"Index":300,"Attempt":0,"Launch Time":1427397600147,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5306,"Index":296,"Attempt":0,"Launch Time":1427397600141,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600148,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":294088,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5311,"Index":301,"Attempt":0,"Launch Time":1427397600148,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5243,"Index":233,"Attempt":0,"Launch Time":1427397600018,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600148,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":113,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":12003053,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5312,"Index":302,"Attempt":0,"Launch Time":1427397600153,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5282,"Index":272,"Attempt":0,"Launch Time":1427397600096,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600153,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":47,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":332701,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5313,"Index":303,"Attempt":0,"Launch Time":1427397600154,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5310,"Index":300,"Attempt":0,"Launch Time":1427397600147,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600154,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":369387,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5314,"Index":304,"Attempt":0,"Launch Time":1427397600156,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5311,"Index":301,"Attempt":0,"Launch Time":1427397600148,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600156,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":358154,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5315,"Index":305,"Attempt":0,"Launch Time":1427397600161,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5313,"Index":303,"Attempt":0,"Launch Time":1427397600154,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600161,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":318287,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5316,"Index":306,"Attempt":0,"Launch Time":1427397600161,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5312,"Index":302,"Attempt":0,"Launch Time":1427397600153,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600162,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":309211,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5317,"Index":307,"Attempt":0,"Launch Time":1427397600163,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5314,"Index":304,"Attempt":0,"Launch Time":1427397600156,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600163,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":322237,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5318,"Index":308,"Attempt":0,"Launch Time":1427397600167,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5315,"Index":305,"Attempt":0,"Launch Time":1427397600161,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600167,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":317661,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5319,"Index":309,"Attempt":0,"Launch Time":1427397600169,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5316,"Index":306,"Attempt":0,"Launch Time":1427397600161,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600169,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":375832,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5320,"Index":310,"Attempt":0,"Launch Time":1427397600172,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5317,"Index":307,"Attempt":0,"Launch Time":1427397600163,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600172,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":352523,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5321,"Index":311,"Attempt":0,"Launch Time":1427397600174,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5318,"Index":308,"Attempt":0,"Launch Time":1427397600167,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600174,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":351413,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5322,"Index":312,"Attempt":0,"Launch Time":1427397600175,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5299,"Index":289,"Attempt":0,"Launch Time":1427397600128,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600175,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":45,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":383606,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5323,"Index":313,"Attempt":0,"Launch Time":1427397600176,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5319,"Index":309,"Attempt":0,"Launch Time":1427397600169,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600176,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":383568,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5324,"Index":314,"Attempt":0,"Launch Time":1427397600176,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5307,"Index":297,"Attempt":0,"Launch Time":1427397600142,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600176,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":33,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":347123,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5325,"Index":315,"Attempt":0,"Launch Time":1427397600181,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5302,"Index":292,"Attempt":0,"Launch Time":1427397600134,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600181,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":33,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":589490,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5326,"Index":316,"Attempt":0,"Launch Time":1427397600193,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5321,"Index":311,"Attempt":0,"Launch Time":1427397600174,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600194,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":19,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":309686,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5327,"Index":317,"Attempt":0,"Launch Time":1427397600194,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5308,"Index":298,"Attempt":0,"Launch Time":1427397600143,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600194,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":51,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":320287,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5328,"Index":318,"Attempt":0,"Launch Time":1427397600200,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5326,"Index":316,"Attempt":0,"Launch Time":1427397600193,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600200,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":449179,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5329,"Index":319,"Attempt":0,"Launch Time":1427397600200,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5327,"Index":317,"Attempt":0,"Launch Time":1427397600194,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600200,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":310490,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5330,"Index":320,"Attempt":0,"Launch Time":1427397600207,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5320,"Index":310,"Attempt":0,"Launch Time":1427397600172,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600207,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":34,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":409070,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5331,"Index":321,"Attempt":0,"Launch Time":1427397600208,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5329,"Index":319,"Attempt":0,"Launch Time":1427397600200,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600208,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":417127,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5332,"Index":322,"Attempt":0,"Launch Time":1427397600209,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5328,"Index":318,"Attempt":0,"Launch Time":1427397600200,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600209,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":339926,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5333,"Index":323,"Attempt":0,"Launch Time":1427397600211,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5324,"Index":314,"Attempt":0,"Launch Time":1427397600176,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600211,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":35,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":4542501,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5334,"Index":324,"Attempt":0,"Launch Time":1427397600214,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5331,"Index":321,"Attempt":0,"Launch Time":1427397600208,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600215,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":313699,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5335,"Index":325,"Attempt":0,"Launch Time":1427397600215,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5330,"Index":320,"Attempt":0,"Launch Time":1427397600207,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600215,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":398043,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5336,"Index":326,"Attempt":0,"Launch Time":1427397600218,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5333,"Index":323,"Attempt":0,"Launch Time":1427397600211,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600219,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1186835,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5337,"Index":327,"Attempt":0,"Launch Time":1427397600221,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5334,"Index":324,"Attempt":0,"Launch Time":1427397600214,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600221,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":298729,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5338,"Index":328,"Attempt":0,"Launch Time":1427397600221,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5325,"Index":315,"Attempt":0,"Launch Time":1427397600181,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600221,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":296449,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5339,"Index":329,"Attempt":0,"Launch Time":1427397600222,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5335,"Index":325,"Attempt":0,"Launch Time":1427397600215,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600222,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":314750,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5340,"Index":330,"Attempt":0,"Launch Time":1427397600225,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5336,"Index":326,"Attempt":0,"Launch Time":1427397600218,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600226,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":286947,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5341,"Index":331,"Attempt":0,"Launch Time":1427397600227,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5337,"Index":327,"Attempt":0,"Launch Time":1427397600221,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600227,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":370885,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5342,"Index":332,"Attempt":0,"Launch Time":1427397600234,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5341,"Index":331,"Attempt":0,"Launch Time":1427397600227,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600234,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":309284,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5343,"Index":333,"Attempt":0,"Launch Time":1427397600241,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5342,"Index":332,"Attempt":0,"Launch Time":1427397600234,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600241,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":326409,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5344,"Index":334,"Attempt":0,"Launch Time":1427397600242,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5340,"Index":330,"Attempt":0,"Launch Time":1427397600225,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600242,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":480980,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5345,"Index":335,"Attempt":0,"Launch Time":1427397600245,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5339,"Index":329,"Attempt":0,"Launch Time":1427397600222,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600245,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":5,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":317991,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5346,"Index":336,"Attempt":0,"Launch Time":1427397600246,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5343,"Index":333,"Attempt":0,"Launch Time":1427397600241,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600247,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":260259,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5347,"Index":337,"Attempt":0,"Launch Time":1427397600249,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5344,"Index":334,"Attempt":0,"Launch Time":1427397600242,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600249,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9666,"Shuffle Write Time":327161,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5348,"Index":338,"Attempt":0,"Launch Time":1427397600253,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5346,"Index":336,"Attempt":0,"Launch Time":1427397600246,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600253,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9648,"Shuffle Write Time":291924,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5349,"Index":339,"Attempt":0,"Launch Time":1427397600253,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5345,"Index":335,"Attempt":0,"Launch Time":1427397600245,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600253,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9650,"Shuffle Write Time":315389,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5350,"Index":340,"Attempt":0,"Launch Time":1427397600266,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5322,"Index":312,"Attempt":0,"Launch Time":1427397600175,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600269,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":88,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":568711,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5351,"Index":341,"Attempt":0,"Launch Time":1427397600270,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5332,"Index":322,"Attempt":0,"Launch Time":1427397600209,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600270,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":53,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1664609,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5352,"Index":342,"Attempt":0,"Launch Time":1427397600270,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5353,"Index":343,"Attempt":0,"Launch Time":1427397600271,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5354,"Index":344,"Attempt":0,"Launch Time":1427397600271,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5309,"Index":299,"Attempt":0,"Launch Time":1427397600143,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600271,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":52,"Executor Run Time":66,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":357299,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5348,"Index":338,"Attempt":0,"Launch Time":1427397600253,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600271,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9651,"Shuffle Write Time":293283,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5355,"Index":345,"Attempt":0,"Launch Time":1427397600272,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5338,"Index":328,"Attempt":0,"Launch Time":1427397600221,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600272,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":472664,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5347,"Index":337,"Attempt":0,"Launch Time":1427397600249,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600272,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9651,"Shuffle Write Time":321184,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5356,"Index":346,"Attempt":0,"Launch Time":1427397600279,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5353,"Index":343,"Attempt":0,"Launch Time":1427397600271,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600279,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":288908,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5357,"Index":347,"Attempt":0,"Launch Time":1427397600280,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5351,"Index":341,"Attempt":0,"Launch Time":1427397600270,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600280,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":332661,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5358,"Index":348,"Attempt":0,"Launch Time":1427397600282,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5355,"Index":345,"Attempt":0,"Launch Time":1427397600272,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600283,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":357449,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5359,"Index":349,"Attempt":0,"Launch Time":1427397600286,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5356,"Index":346,"Attempt":0,"Launch Time":1427397600279,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600287,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":345216,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5360,"Index":350,"Attempt":0,"Launch Time":1427397600288,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5357,"Index":347,"Attempt":0,"Launch Time":1427397600280,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600288,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":336248,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5361,"Index":351,"Attempt":0,"Launch Time":1427397600289,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5323,"Index":313,"Attempt":0,"Launch Time":1427397600176,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600289,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":60,"Executor Run Time":32,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":3545023,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5362,"Index":352,"Attempt":0,"Launch Time":1427397600289,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5358,"Index":348,"Attempt":0,"Launch Time":1427397600282,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600290,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":332685,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5363,"Index":353,"Attempt":0,"Launch Time":1427397600294,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5360,"Index":350,"Attempt":0,"Launch Time":1427397600288,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600295,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":343810,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5364,"Index":354,"Attempt":0,"Launch Time":1427397600295,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5359,"Index":349,"Attempt":0,"Launch Time":1427397600286,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600295,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":350327,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5365,"Index":355,"Attempt":0,"Launch Time":1427397600296,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5362,"Index":352,"Attempt":0,"Launch Time":1427397600289,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600296,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":299354,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5366,"Index":356,"Attempt":0,"Launch Time":1427397600297,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5361,"Index":351,"Attempt":0,"Launch Time":1427397600289,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600297,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":331606,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5367,"Index":357,"Attempt":0,"Launch Time":1427397600303,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5365,"Index":355,"Attempt":0,"Launch Time":1427397600296,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600303,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":323827,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5368,"Index":358,"Attempt":0,"Launch Time":1427397600305,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5369,"Index":359,"Attempt":0,"Launch Time":1427397600305,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5370,"Index":360,"Attempt":0,"Launch Time":1427397600306,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5371,"Index":361,"Attempt":0,"Launch Time":1427397600306,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5366,"Index":356,"Attempt":0,"Launch Time":1427397600297,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600306,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":323674,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5364,"Index":354,"Attempt":0,"Launch Time":1427397600295,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600306,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":1111118,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5363,"Index":353,"Attempt":0,"Launch Time":1427397600294,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600306,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":326137,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5352,"Index":342,"Attempt":0,"Launch Time":1427397600270,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600306,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":296595,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5372,"Index":362,"Attempt":0,"Launch Time":1427397600319,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5368,"Index":358,"Attempt":0,"Launch Time":1427397600305,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600319,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":337630,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5373,"Index":363,"Attempt":0,"Launch Time":1427397600320,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5371,"Index":361,"Attempt":0,"Launch Time":1427397600306,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600320,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":319045,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5374,"Index":364,"Attempt":0,"Launch Time":1427397600320,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5369,"Index":359,"Attempt":0,"Launch Time":1427397600305,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600320,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":373076,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5375,"Index":365,"Attempt":0,"Launch Time":1427397600321,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5349,"Index":339,"Attempt":0,"Launch Time":1427397600253,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600321,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":13,"Executor Run Time":55,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9647,"Shuffle Write Time":623048,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5376,"Index":366,"Attempt":0,"Launch Time":1427397600323,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5367,"Index":357,"Attempt":0,"Launch Time":1427397600303,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600323,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":331621,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5377,"Index":367,"Attempt":0,"Launch Time":1427397600326,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5372,"Index":362,"Attempt":0,"Launch Time":1427397600319,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600326,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":307594,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5378,"Index":368,"Attempt":0,"Launch Time":1427397600327,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5379,"Index":369,"Attempt":0,"Launch Time":1427397600327,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5374,"Index":364,"Attempt":0,"Launch Time":1427397600320,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600327,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":344447,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5373,"Index":363,"Attempt":0,"Launch Time":1427397600320,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600327,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":315318,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5380,"Index":370,"Attempt":0,"Launch Time":1427397600330,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5370,"Index":360,"Attempt":0,"Launch Time":1427397600306,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600330,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":23,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1015849,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5381,"Index":371,"Attempt":0,"Launch Time":1427397600331,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5376,"Index":366,"Attempt":0,"Launch Time":1427397600323,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600331,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":435803,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5382,"Index":372,"Attempt":0,"Launch Time":1427397600333,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5377,"Index":367,"Attempt":0,"Launch Time":1427397600326,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600333,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":295740,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5383,"Index":373,"Attempt":0,"Launch Time":1427397600335,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5379,"Index":369,"Attempt":0,"Launch Time":1427397600327,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600336,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":296648,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5384,"Index":374,"Attempt":0,"Launch Time":1427397600336,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5380,"Index":370,"Attempt":0,"Launch Time":1427397600330,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600336,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":308464,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5385,"Index":375,"Attempt":0,"Launch Time":1427397600338,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5386,"Index":376,"Attempt":0,"Launch Time":1427397600338,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5381,"Index":371,"Attempt":0,"Launch Time":1427397600331,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600338,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":313683,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5378,"Index":368,"Attempt":0,"Launch Time":1427397600327,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600338,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":331629,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5387,"Index":377,"Attempt":0,"Launch Time":1427397600341,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5382,"Index":372,"Attempt":0,"Launch Time":1427397600333,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600341,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":299596,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5388,"Index":378,"Attempt":0,"Launch Time":1427397600343,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5383,"Index":373,"Attempt":0,"Launch Time":1427397600335,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600343,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":262532,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5389,"Index":379,"Attempt":0,"Launch Time":1427397600343,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5384,"Index":374,"Attempt":0,"Launch Time":1427397600336,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600343,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9612,"Shuffle Write Time":294948,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5390,"Index":380,"Attempt":0,"Launch Time":1427397600346,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5386,"Index":376,"Attempt":0,"Launch Time":1427397600338,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600346,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":355394,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5391,"Index":381,"Attempt":0,"Launch Time":1427397600348,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5385,"Index":375,"Attempt":0,"Launch Time":1427397600338,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600348,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":453212,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5392,"Index":382,"Attempt":0,"Launch Time":1427397600351,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5389,"Index":379,"Attempt":0,"Launch Time":1427397600343,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600351,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":326506,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5393,"Index":383,"Attempt":0,"Launch Time":1427397600352,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5388,"Index":378,"Attempt":0,"Launch Time":1427397600343,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600352,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":403682,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5394,"Index":384,"Attempt":0,"Launch Time":1427397600354,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5390,"Index":380,"Attempt":0,"Launch Time":1427397600346,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600354,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":319922,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5395,"Index":385,"Attempt":0,"Launch Time":1427397600355,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5387,"Index":377,"Attempt":0,"Launch Time":1427397600341,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600355,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":361570,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5396,"Index":386,"Attempt":0,"Launch Time":1427397600359,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5392,"Index":382,"Attempt":0,"Launch Time":1427397600351,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600359,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9614,"Shuffle Write Time":318437,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5397,"Index":387,"Attempt":0,"Launch Time":1427397600360,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5393,"Index":383,"Attempt":0,"Launch Time":1427397600352,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600360,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":347668,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5398,"Index":388,"Attempt":0,"Launch Time":1427397600362,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5375,"Index":365,"Attempt":0,"Launch Time":1427397600321,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600362,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":13,"Executor Run Time":22,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":322644,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5399,"Index":389,"Attempt":0,"Launch Time":1427397600364,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5391,"Index":381,"Attempt":0,"Launch Time":1427397600348,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600364,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":382391,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5400,"Index":390,"Attempt":0,"Launch Time":1427397600368,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5394,"Index":384,"Attempt":0,"Launch Time":1427397600354,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600368,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":393478,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5401,"Index":391,"Attempt":0,"Launch Time":1427397600368,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5395,"Index":385,"Attempt":0,"Launch Time":1427397600355,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600369,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":399273,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5402,"Index":392,"Attempt":0,"Launch Time":1427397600370,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5403,"Index":393,"Attempt":0,"Launch Time":1427397600370,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5397,"Index":387,"Attempt":0,"Launch Time":1427397600360,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600370,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":394729,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5396,"Index":386,"Attempt":0,"Launch Time":1427397600359,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600370,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":944783,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5404,"Index":394,"Attempt":0,"Launch Time":1427397600372,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5399,"Index":389,"Attempt":0,"Launch Time":1427397600364,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600372,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":590343,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5405,"Index":395,"Attempt":0,"Launch Time":1427397600376,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5400,"Index":390,"Attempt":0,"Launch Time":1427397600368,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600376,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9615,"Shuffle Write Time":339710,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5406,"Index":396,"Attempt":0,"Launch Time":1427397600377,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5401,"Index":391,"Attempt":0,"Launch Time":1427397600368,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600377,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":512986,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5407,"Index":397,"Attempt":0,"Launch Time":1427397600379,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5403,"Index":393,"Attempt":0,"Launch Time":1427397600370,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600379,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":303678,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5408,"Index":398,"Attempt":0,"Launch Time":1427397600379,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5409,"Index":399,"Attempt":0,"Launch Time":1427397600379,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5404,"Index":394,"Attempt":0,"Launch Time":1427397600372,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600379,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":318530,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5402,"Index":392,"Attempt":0,"Launch Time":1427397600370,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600380,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":309311,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5410,"Index":400,"Attempt":0,"Launch Time":1427397600384,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5405,"Index":395,"Attempt":0,"Launch Time":1427397600376,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600384,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":289736,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5411,"Index":401,"Attempt":0,"Launch Time":1427397600385,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5406,"Index":396,"Attempt":0,"Launch Time":1427397600377,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600385,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":360320,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5412,"Index":402,"Attempt":0,"Launch Time":1427397600386,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5407,"Index":397,"Attempt":0,"Launch Time":1427397600379,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600386,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":288817,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5413,"Index":403,"Attempt":0,"Launch Time":1427397600387,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5414,"Index":404,"Attempt":0,"Launch Time":1427397600387,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5409,"Index":399,"Attempt":0,"Launch Time":1427397600379,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600387,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":272356,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5354,"Index":344,"Attempt":0,"Launch Time":1427397600271,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600387,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":17,"Executor Run Time":48,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1598398,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5415,"Index":405,"Attempt":0,"Launch Time":1427397600389,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5408,"Index":398,"Attempt":0,"Launch Time":1427397600379,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600390,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":292343,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5416,"Index":406,"Attempt":0,"Launch Time":1427397600392,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5410,"Index":400,"Attempt":0,"Launch Time":1427397600384,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600392,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":307210,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5417,"Index":407,"Attempt":0,"Launch Time":1427397600407,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5398,"Index":388,"Attempt":0,"Launch Time":1427397600362,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600407,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":29,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":305432,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5418,"Index":408,"Attempt":0,"Launch Time":1427397600410,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5350,"Index":340,"Attempt":0,"Launch Time":1427397600266,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600410,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":83,"Executor Run Time":7,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9649,"Shuffle Write Time":347891,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5419,"Index":409,"Attempt":0,"Launch Time":1427397600410,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5413,"Index":403,"Attempt":0,"Launch Time":1427397600387,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600411,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":14,"Executor Run Time":7,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":317162,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5420,"Index":410,"Attempt":0,"Launch Time":1427397600417,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5417,"Index":407,"Attempt":0,"Launch Time":1427397600407,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600417,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":388575,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5421,"Index":411,"Attempt":0,"Launch Time":1427397600418,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5419,"Index":409,"Attempt":0,"Launch Time":1427397600410,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600418,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":360589,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5422,"Index":412,"Attempt":0,"Launch Time":1427397600418,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5418,"Index":408,"Attempt":0,"Launch Time":1427397600410,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600419,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":532993,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5423,"Index":413,"Attempt":0,"Launch Time":1427397600426,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5422,"Index":412,"Attempt":0,"Launch Time":1427397600418,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600426,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":317068,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5424,"Index":414,"Attempt":0,"Launch Time":1427397600426,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5420,"Index":410,"Attempt":0,"Launch Time":1427397600417,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600426,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":296431,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5425,"Index":415,"Attempt":0,"Launch Time":1427397600426,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5421,"Index":411,"Attempt":0,"Launch Time":1427397600418,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600427,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":323191,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5426,"Index":416,"Attempt":0,"Launch Time":1427397600433,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5423,"Index":413,"Attempt":0,"Launch Time":1427397600426,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600433,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":306320,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5427,"Index":417,"Attempt":0,"Launch Time":1427397600433,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5424,"Index":414,"Attempt":0,"Launch Time":1427397600426,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600433,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":286302,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5428,"Index":418,"Attempt":0,"Launch Time":1427397600433,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5425,"Index":415,"Attempt":0,"Launch Time":1427397600426,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600433,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":281605,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5429,"Index":419,"Attempt":0,"Launch Time":1427397600439,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5426,"Index":416,"Attempt":0,"Launch Time":1427397600433,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600439,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":323159,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5430,"Index":420,"Attempt":0,"Launch Time":1427397600440,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5427,"Index":417,"Attempt":0,"Launch Time":1427397600433,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600440,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":340210,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5431,"Index":421,"Attempt":0,"Launch Time":1427397600440,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5428,"Index":418,"Attempt":0,"Launch Time":1427397600433,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600440,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":423926,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5432,"Index":422,"Attempt":0,"Launch Time":1427397600445,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5429,"Index":419,"Attempt":0,"Launch Time":1427397600439,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600446,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":309757,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5433,"Index":423,"Attempt":0,"Launch Time":1427397600446,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5430,"Index":420,"Attempt":0,"Launch Time":1427397600440,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600446,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":329550,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5434,"Index":424,"Attempt":0,"Launch Time":1427397600447,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5431,"Index":421,"Attempt":0,"Launch Time":1427397600440,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600447,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":332303,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5435,"Index":425,"Attempt":0,"Launch Time":1427397600452,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5432,"Index":422,"Attempt":0,"Launch Time":1427397600445,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600452,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":309644,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5436,"Index":426,"Attempt":0,"Launch Time":1427397600453,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5433,"Index":423,"Attempt":0,"Launch Time":1427397600446,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600453,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":327459,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5437,"Index":427,"Attempt":0,"Launch Time":1427397600454,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5434,"Index":424,"Attempt":0,"Launch Time":1427397600447,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600454,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":326269,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5438,"Index":428,"Attempt":0,"Launch Time":1427397600459,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5435,"Index":425,"Attempt":0,"Launch Time":1427397600452,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600459,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":309488,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5439,"Index":429,"Attempt":0,"Launch Time":1427397600460,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5436,"Index":426,"Attempt":0,"Launch Time":1427397600453,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600460,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":320789,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5440,"Index":430,"Attempt":0,"Launch Time":1427397600461,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5437,"Index":427,"Attempt":0,"Launch Time":1427397600454,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600461,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":340055,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5441,"Index":431,"Attempt":0,"Launch Time":1427397600462,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5412,"Index":402,"Attempt":0,"Launch Time":1427397600386,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600462,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":76,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":982436,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5442,"Index":432,"Attempt":0,"Launch Time":1427397600467,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5438,"Index":428,"Attempt":0,"Launch Time":1427397600459,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600467,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":369914,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5443,"Index":433,"Attempt":0,"Launch Time":1427397600468,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5439,"Index":429,"Attempt":0,"Launch Time":1427397600460,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600468,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":373651,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5444,"Index":434,"Attempt":0,"Launch Time":1427397600470,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5445,"Index":435,"Attempt":0,"Launch Time":1427397600470,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5440,"Index":430,"Attempt":0,"Launch Time":1427397600461,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600471,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":452749,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5414,"Index":404,"Attempt":0,"Launch Time":1427397600387,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600471,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":83,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":352650,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5446,"Index":436,"Attempt":0,"Launch Time":1427397600474,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5442,"Index":432,"Attempt":0,"Launch Time":1427397600467,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600474,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":337612,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5447,"Index":437,"Attempt":0,"Launch Time":1427397600475,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5443,"Index":433,"Attempt":0,"Launch Time":1427397600468,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600475,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":380711,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5448,"Index":438,"Attempt":0,"Launch Time":1427397600477,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5444,"Index":434,"Attempt":0,"Launch Time":1427397600470,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600477,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":340148,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5449,"Index":439,"Attempt":0,"Launch Time":1427397600479,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5445,"Index":435,"Attempt":0,"Launch Time":1427397600470,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600480,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1029239,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5450,"Index":440,"Attempt":0,"Launch Time":1427397600480,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5446,"Index":436,"Attempt":0,"Launch Time":1427397600474,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600480,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":305255,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5451,"Index":441,"Attempt":0,"Launch Time":1427397600482,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5447,"Index":437,"Attempt":0,"Launch Time":1427397600475,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600482,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":301247,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5452,"Index":442,"Attempt":0,"Launch Time":1427397600483,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5448,"Index":438,"Attempt":0,"Launch Time":1427397600477,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600483,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":310076,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5453,"Index":443,"Attempt":0,"Launch Time":1427397600486,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5450,"Index":440,"Attempt":0,"Launch Time":1427397600480,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600486,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":307069,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5454,"Index":444,"Attempt":0,"Launch Time":1427397600489,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5451,"Index":441,"Attempt":0,"Launch Time":1427397600482,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600489,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":324180,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5455,"Index":445,"Attempt":0,"Launch Time":1427397600489,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5452,"Index":442,"Attempt":0,"Launch Time":1427397600483,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600489,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":299513,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5456,"Index":446,"Attempt":0,"Launch Time":1427397600493,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5453,"Index":443,"Attempt":0,"Launch Time":1427397600486,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600493,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":284443,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5457,"Index":447,"Attempt":0,"Launch Time":1427397600496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5454,"Index":444,"Attempt":0,"Launch Time":1427397600489,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600496,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":329378,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5458,"Index":448,"Attempt":0,"Launch Time":1427397600496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5455,"Index":445,"Attempt":0,"Launch Time":1427397600489,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600496,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9605,"Shuffle Write Time":326806,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5459,"Index":449,"Attempt":0,"Launch Time":1427397600500,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5456,"Index":446,"Attempt":0,"Launch Time":1427397600493,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600500,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":287848,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5460,"Index":450,"Attempt":0,"Launch Time":1427397600515,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5411,"Index":401,"Attempt":0,"Launch Time":1427397600385,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600515,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":128,"Result Size":930,"JVM GC Time":14,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":74684537,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5461,"Index":451,"Attempt":0,"Launch Time":1427397600516,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5459,"Index":449,"Attempt":0,"Launch Time":1427397600500,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600516,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":314348,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5462,"Index":452,"Attempt":0,"Launch Time":1427397600518,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5457,"Index":447,"Attempt":0,"Launch Time":1427397600496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600518,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":22,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":362949,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5463,"Index":453,"Attempt":0,"Launch Time":1427397600518,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5458,"Index":448,"Attempt":0,"Launch Time":1427397600496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600518,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":22,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":328339,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5464,"Index":454,"Attempt":0,"Launch Time":1427397600521,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5415,"Index":405,"Attempt":0,"Launch Time":1427397600389,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600521,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":130,"Result Size":930,"JVM GC Time":14,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":377101,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5465,"Index":455,"Attempt":0,"Launch Time":1427397600521,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5449,"Index":439,"Attempt":0,"Launch Time":1427397600479,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600521,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":41,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":452437,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5466,"Index":456,"Attempt":0,"Launch Time":1427397600526,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5467,"Index":457,"Attempt":0,"Launch Time":1427397600526,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5416,"Index":406,"Attempt":0,"Launch Time":1427397600392,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600527,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":11,"Executor Run Time":123,"Result Size":930,"JVM GC Time":14,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":365240,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5441,"Index":431,"Attempt":0,"Launch Time":1427397600462,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600527,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":22,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1418167,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5468,"Index":458,"Attempt":0,"Launch Time":1427397600529,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5460,"Index":450,"Attempt":0,"Launch Time":1427397600515,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600529,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":886998,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5469,"Index":459,"Attempt":0,"Launch Time":1427397600530,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5464,"Index":454,"Attempt":0,"Launch Time":1427397600521,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600530,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1238234,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5470,"Index":460,"Attempt":0,"Launch Time":1427397600535,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5471,"Index":461,"Attempt":0,"Launch Time":1427397600535,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5467,"Index":457,"Attempt":0,"Launch Time":1427397600526,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600535,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":328704,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5468,"Index":458,"Attempt":0,"Launch Time":1427397600529,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600535,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":271657,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5472,"Index":462,"Attempt":0,"Launch Time":1427397600536,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5469,"Index":459,"Attempt":0,"Launch Time":1427397600530,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600536,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":306439,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5473,"Index":463,"Attempt":0,"Launch Time":1427397600541,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5470,"Index":460,"Attempt":0,"Launch Time":1427397600535,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600541,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":267939,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5474,"Index":464,"Attempt":0,"Launch Time":1427397600542,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5471,"Index":461,"Attempt":0,"Launch Time":1427397600535,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600542,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":303188,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5475,"Index":465,"Attempt":0,"Launch Time":1427397600543,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5472,"Index":462,"Attempt":0,"Launch Time":1427397600536,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600543,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":317081,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5476,"Index":466,"Attempt":0,"Launch Time":1427397600549,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5473,"Index":463,"Attempt":0,"Launch Time":1427397600541,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600549,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9607,"Shuffle Write Time":406927,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5477,"Index":467,"Attempt":0,"Launch Time":1427397600552,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5474,"Index":464,"Attempt":0,"Launch Time":1427397600542,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600552,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":318878,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5478,"Index":468,"Attempt":0,"Launch Time":1427397600554,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5475,"Index":465,"Attempt":0,"Launch Time":1427397600543,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600554,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":400941,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5479,"Index":469,"Attempt":0,"Launch Time":1427397600557,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5476,"Index":466,"Attempt":0,"Launch Time":1427397600549,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600557,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":366701,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5480,"Index":470,"Attempt":0,"Launch Time":1427397600560,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5478,"Index":468,"Attempt":0,"Launch Time":1427397600554,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600560,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":303974,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5481,"Index":471,"Attempt":0,"Launch Time":1427397600561,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5477,"Index":467,"Attempt":0,"Launch Time":1427397600552,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600561,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":331645,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5482,"Index":472,"Attempt":0,"Launch Time":1427397600564,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5479,"Index":469,"Attempt":0,"Launch Time":1427397600557,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600564,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":270620,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5483,"Index":473,"Attempt":0,"Launch Time":1427397600567,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5480,"Index":470,"Attempt":0,"Launch Time":1427397600560,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600567,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":327542,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5484,"Index":474,"Attempt":0,"Launch Time":1427397600568,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5481,"Index":471,"Attempt":0,"Launch Time":1427397600561,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600568,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9613,"Shuffle Write Time":328893,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5485,"Index":475,"Attempt":0,"Launch Time":1427397600571,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5482,"Index":472,"Attempt":0,"Launch Time":1427397600564,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600572,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":310674,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5486,"Index":476,"Attempt":0,"Launch Time":1427397600573,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5483,"Index":473,"Attempt":0,"Launch Time":1427397600567,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600573,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":302095,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5487,"Index":477,"Attempt":0,"Launch Time":1427397600575,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5484,"Index":474,"Attempt":0,"Launch Time":1427397600568,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600575,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":262146,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5488,"Index":478,"Attempt":0,"Launch Time":1427397600578,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5485,"Index":475,"Attempt":0,"Launch Time":1427397600571,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600578,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":264853,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5489,"Index":479,"Attempt":0,"Launch Time":1427397600579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5486,"Index":476,"Attempt":0,"Launch Time":1427397600573,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600580,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":319588,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5490,"Index":480,"Attempt":0,"Launch Time":1427397600581,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5487,"Index":477,"Attempt":0,"Launch Time":1427397600575,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600581,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":289320,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5491,"Index":481,"Attempt":0,"Launch Time":1427397600585,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5489,"Index":479,"Attempt":0,"Launch Time":1427397600579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600585,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":250921,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5492,"Index":482,"Attempt":0,"Launch Time":1427397600586,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5488,"Index":478,"Attempt":0,"Launch Time":1427397600578,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600586,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":265215,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5493,"Index":483,"Attempt":0,"Launch Time":1427397600588,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5490,"Index":480,"Attempt":0,"Launch Time":1427397600581,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600588,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":286364,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5494,"Index":484,"Attempt":0,"Launch Time":1427397600591,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5491,"Index":481,"Attempt":0,"Launch Time":1427397600585,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600591,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":269705,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5495,"Index":485,"Attempt":0,"Launch Time":1427397600593,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5492,"Index":482,"Attempt":0,"Launch Time":1427397600586,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600594,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":304376,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5496,"Index":486,"Attempt":0,"Launch Time":1427397600594,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5493,"Index":483,"Attempt":0,"Launch Time":1427397600588,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600594,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":305569,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5497,"Index":487,"Attempt":0,"Launch Time":1427397600597,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5494,"Index":484,"Attempt":0,"Launch Time":1427397600591,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600597,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":247407,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5498,"Index":488,"Attempt":0,"Launch Time":1427397600601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5495,"Index":485,"Attempt":0,"Launch Time":1427397600593,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600601,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":339427,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5499,"Index":489,"Attempt":0,"Launch Time":1427397600601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5496,"Index":486,"Attempt":0,"Launch Time":1427397600594,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600602,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":347355,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5500,"Index":490,"Attempt":0,"Launch Time":1427397600604,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5497,"Index":487,"Attempt":0,"Launch Time":1427397600597,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600604,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":353405,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5501,"Index":491,"Attempt":0,"Launch Time":1427397600609,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5499,"Index":489,"Attempt":0,"Launch Time":1427397600601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600609,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":355103,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5502,"Index":492,"Attempt":0,"Launch Time":1427397600609,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5498,"Index":488,"Attempt":0,"Launch Time":1427397600601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600609,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":322798,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5503,"Index":493,"Attempt":0,"Launch Time":1427397600610,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5500,"Index":490,"Attempt":0,"Launch Time":1427397600604,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600610,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":306803,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5504,"Index":494,"Attempt":0,"Launch Time":1427397600615,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5501,"Index":491,"Attempt":0,"Launch Time":1427397600609,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600615,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":287719,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5505,"Index":495,"Attempt":0,"Launch Time":1427397600616,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5502,"Index":492,"Attempt":0,"Launch Time":1427397600609,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600616,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":307727,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5506,"Index":496,"Attempt":0,"Launch Time":1427397600617,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5503,"Index":493,"Attempt":0,"Launch Time":1427397600610,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600617,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":309248,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5507,"Index":497,"Attempt":0,"Launch Time":1427397600618,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5466,"Index":456,"Attempt":0,"Launch Time":1427397600526,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600618,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":89,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":300073,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5508,"Index":498,"Attempt":0,"Launch Time":1427397600627,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5504,"Index":494,"Attempt":0,"Launch Time":1427397600615,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600627,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":302455,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5509,"Index":499,"Attempt":0,"Launch Time":1427397600631,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5465,"Index":455,"Attempt":0,"Launch Time":1427397600521,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600631,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":108,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":318366,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5510,"Index":500,"Attempt":0,"Launch Time":1427397600632,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5506,"Index":496,"Attempt":0,"Launch Time":1427397600617,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600632,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":375647,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5511,"Index":501,"Attempt":0,"Launch Time":1427397600634,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5508,"Index":498,"Attempt":0,"Launch Time":1427397600627,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600634,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":290952,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5512,"Index":502,"Attempt":0,"Launch Time":1427397600637,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5509,"Index":499,"Attempt":0,"Launch Time":1427397600631,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600637,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":298579,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5513,"Index":503,"Attempt":0,"Launch Time":1427397600639,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5510,"Index":500,"Attempt":0,"Launch Time":1427397600632,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600639,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":308073,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5514,"Index":504,"Attempt":0,"Launch Time":1427397600640,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5511,"Index":501,"Attempt":0,"Launch Time":1427397600634,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600641,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":277358,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5515,"Index":505,"Attempt":0,"Launch Time":1427397600644,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5512,"Index":502,"Attempt":0,"Launch Time":1427397600637,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600644,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":318419,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5516,"Index":506,"Attempt":0,"Launch Time":1427397600645,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5513,"Index":503,"Attempt":0,"Launch Time":1427397600639,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600646,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":306978,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5517,"Index":507,"Attempt":0,"Launch Time":1427397600647,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5518,"Index":508,"Attempt":0,"Launch Time":1427397600647,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5514,"Index":504,"Attempt":0,"Launch Time":1427397600640,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600648,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":315632,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5462,"Index":452,"Attempt":0,"Launch Time":1427397600518,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600648,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":128,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":306075,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5519,"Index":509,"Attempt":0,"Launch Time":1427397600651,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5461,"Index":451,"Attempt":0,"Launch Time":1427397600516,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600651,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":134,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":413676,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5520,"Index":510,"Attempt":0,"Launch Time":1427397600651,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5515,"Index":505,"Attempt":0,"Launch Time":1427397600644,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600651,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":375360,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5521,"Index":511,"Attempt":0,"Launch Time":1427397600653,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5463,"Index":453,"Attempt":0,"Launch Time":1427397600518,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600653,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":134,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":348853,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5522,"Index":512,"Attempt":0,"Launch Time":1427397600656,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5523,"Index":513,"Attempt":0,"Launch Time":1427397600657,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5517,"Index":507,"Attempt":0,"Launch Time":1427397600647,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600657,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":376807,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5507,"Index":497,"Attempt":0,"Launch Time":1427397600618,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600657,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":28,"Executor Run Time":9,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":692998,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5524,"Index":514,"Attempt":0,"Launch Time":1427397600658,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5519,"Index":509,"Attempt":0,"Launch Time":1427397600651,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600658,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":370889,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5525,"Index":515,"Attempt":0,"Launch Time":1427397600659,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5520,"Index":510,"Attempt":0,"Launch Time":1427397600651,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600659,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":336940,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5526,"Index":516,"Attempt":0,"Launch Time":1427397600660,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5521,"Index":511,"Attempt":0,"Launch Time":1427397600653,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600661,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":301581,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5527,"Index":517,"Attempt":0,"Launch Time":1427397600667,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5522,"Index":512,"Attempt":0,"Launch Time":1427397600656,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600667,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":322468,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5528,"Index":518,"Attempt":0,"Launch Time":1427397600668,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5525,"Index":515,"Attempt":0,"Launch Time":1427397600659,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600668,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":325301,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5529,"Index":519,"Attempt":0,"Launch Time":1427397600669,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5526,"Index":516,"Attempt":0,"Launch Time":1427397600660,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600669,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":361020,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5530,"Index":520,"Attempt":0,"Launch Time":1427397600673,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5527,"Index":517,"Attempt":0,"Launch Time":1427397600667,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600673,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":329988,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5531,"Index":521,"Attempt":0,"Launch Time":1427397600674,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5523,"Index":513,"Attempt":0,"Launch Time":1427397600657,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600674,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":353984,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5532,"Index":522,"Attempt":0,"Launch Time":1427397600677,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5524,"Index":514,"Attempt":0,"Launch Time":1427397600658,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600677,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":406040,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5533,"Index":523,"Attempt":0,"Launch Time":1427397600680,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5534,"Index":524,"Attempt":0,"Launch Time":1427397600680,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5530,"Index":520,"Attempt":0,"Launch Time":1427397600673,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600680,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":320119,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5529,"Index":519,"Attempt":0,"Launch Time":1427397600669,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600680,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":313739,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5535,"Index":525,"Attempt":0,"Launch Time":1427397600681,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5531,"Index":521,"Attempt":0,"Launch Time":1427397600674,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600681,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":319439,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5536,"Index":526,"Attempt":0,"Launch Time":1427397600684,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5528,"Index":518,"Attempt":0,"Launch Time":1427397600668,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600684,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":331628,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5537,"Index":527,"Attempt":0,"Launch Time":1427397600687,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5534,"Index":524,"Attempt":0,"Launch Time":1427397600680,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600687,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":306707,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5538,"Index":528,"Attempt":0,"Launch Time":1427397600689,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5539,"Index":529,"Attempt":0,"Launch Time":1427397600689,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5533,"Index":523,"Attempt":0,"Launch Time":1427397600680,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600689,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":324276,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5535,"Index":525,"Attempt":0,"Launch Time":1427397600681,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600689,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":313639,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5540,"Index":530,"Attempt":0,"Launch Time":1427397600690,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5536,"Index":526,"Attempt":0,"Launch Time":1427397600684,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600690,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":281798,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5541,"Index":531,"Attempt":0,"Launch Time":1427397600693,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5532,"Index":522,"Attempt":0,"Launch Time":1427397600677,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600693,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":348913,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5542,"Index":532,"Attempt":0,"Launch Time":1427397600694,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5537,"Index":527,"Attempt":0,"Launch Time":1427397600687,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600694,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":321481,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5543,"Index":533,"Attempt":0,"Launch Time":1427397600695,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5538,"Index":528,"Attempt":0,"Launch Time":1427397600689,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600695,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":285832,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5544,"Index":534,"Attempt":0,"Launch Time":1427397600699,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5540,"Index":530,"Attempt":0,"Launch Time":1427397600690,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600699,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":357646,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5545,"Index":535,"Attempt":0,"Launch Time":1427397600699,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5539,"Index":529,"Attempt":0,"Launch Time":1427397600689,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600699,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":620315,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5546,"Index":536,"Attempt":0,"Launch Time":1427397600700,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5541,"Index":531,"Attempt":0,"Launch Time":1427397600693,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600700,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":386499,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5547,"Index":537,"Attempt":0,"Launch Time":1427397600703,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5542,"Index":532,"Attempt":0,"Launch Time":1427397600694,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600703,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":460049,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5548,"Index":538,"Attempt":0,"Launch Time":1427397600703,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5543,"Index":533,"Attempt":0,"Launch Time":1427397600695,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600703,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":372802,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5549,"Index":539,"Attempt":0,"Launch Time":1427397600707,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5545,"Index":535,"Attempt":0,"Launch Time":1427397600699,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600707,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":343070,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5550,"Index":540,"Attempt":0,"Launch Time":1427397600708,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5544,"Index":534,"Attempt":0,"Launch Time":1427397600699,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600708,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":332011,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5551,"Index":541,"Attempt":0,"Launch Time":1427397600711,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5548,"Index":538,"Attempt":0,"Launch Time":1427397600703,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600711,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":304894,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5552,"Index":542,"Attempt":0,"Launch Time":1427397600711,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5553,"Index":543,"Attempt":0,"Launch Time":1427397600719,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5549,"Index":539,"Attempt":0,"Launch Time":1427397600707,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600719,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":279532,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5554,"Index":544,"Attempt":0,"Launch Time":1427397600725,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5552,"Index":542,"Attempt":0,"Launch Time":1427397600711,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600726,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":314359,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5555,"Index":545,"Attempt":0,"Launch Time":1427397600726,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5553,"Index":543,"Attempt":0,"Launch Time":1427397600719,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600726,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":298472,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5556,"Index":546,"Attempt":0,"Launch Time":1427397600732,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5554,"Index":544,"Attempt":0,"Launch Time":1427397600725,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600732,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":341941,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5557,"Index":547,"Attempt":0,"Launch Time":1427397600732,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5555,"Index":545,"Attempt":0,"Launch Time":1427397600726,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600732,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":297617,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5558,"Index":548,"Attempt":0,"Launch Time":1427397600739,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5557,"Index":547,"Attempt":0,"Launch Time":1427397600732,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600739,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":316231,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5559,"Index":549,"Attempt":0,"Launch Time":1427397600739,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5556,"Index":546,"Attempt":0,"Launch Time":1427397600732,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600739,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":368342,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5560,"Index":550,"Attempt":0,"Launch Time":1427397600746,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5558,"Index":548,"Attempt":0,"Launch Time":1427397600739,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600746,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":328645,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5547,"Index":537,"Attempt":0,"Launch Time":1427397600703,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600750,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":310066,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5561,"Index":551,"Attempt":0,"Launch Time":1427397600761,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5550,"Index":540,"Attempt":0,"Launch Time":1427397600708,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600761,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":48,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":372575,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5562,"Index":552,"Attempt":0,"Launch Time":1427397600761,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5551,"Index":541,"Attempt":0,"Launch Time":1427397600711,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600761,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":46,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":503345,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5563,"Index":553,"Attempt":0,"Launch Time":1427397600761,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5546,"Index":536,"Attempt":0,"Launch Time":1427397600700,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600761,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":54,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":519886,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5564,"Index":554,"Attempt":0,"Launch Time":1427397600762,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5560,"Index":550,"Attempt":0,"Launch Time":1427397600746,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600762,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":335914,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5565,"Index":555,"Attempt":0,"Launch Time":1427397600762,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5505,"Index":495,"Attempt":0,"Launch Time":1427397600616,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600762,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":133,"Result Size":930,"JVM GC Time":11,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":384212,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5566,"Index":556,"Attempt":0,"Launch Time":1427397600762,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5559,"Index":549,"Attempt":0,"Launch Time":1427397600739,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600762,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":518178,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5567,"Index":557,"Attempt":0,"Launch Time":1427397600768,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5563,"Index":553,"Attempt":0,"Launch Time":1427397600761,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600768,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":314169,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5568,"Index":558,"Attempt":0,"Launch Time":1427397600769,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5565,"Index":555,"Attempt":0,"Launch Time":1427397600762,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600769,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":311372,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5569,"Index":559,"Attempt":0,"Launch Time":1427397600770,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5561,"Index":551,"Attempt":0,"Launch Time":1427397600761,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600770,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":322471,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5570,"Index":560,"Attempt":0,"Launch Time":1427397600771,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5564,"Index":554,"Attempt":0,"Launch Time":1427397600762,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600771,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":310239,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5571,"Index":561,"Attempt":0,"Launch Time":1427397600776,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5568,"Index":558,"Attempt":0,"Launch Time":1427397600769,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600776,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":401332,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5572,"Index":562,"Attempt":0,"Launch Time":1427397600777,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5562,"Index":552,"Attempt":0,"Launch Time":1427397600761,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600777,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":340991,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5573,"Index":563,"Attempt":0,"Launch Time":1427397600782,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5571,"Index":561,"Attempt":0,"Launch Time":1427397600776,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600782,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":281853,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5574,"Index":564,"Attempt":0,"Launch Time":1427397600785,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5572,"Index":562,"Attempt":0,"Launch Time":1427397600777,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600785,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":310066,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5575,"Index":565,"Attempt":0,"Launch Time":1427397600788,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5518,"Index":508,"Attempt":0,"Launch Time":1427397600647,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600788,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":9,"Executor Run Time":29,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":311336,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5576,"Index":566,"Attempt":0,"Launch Time":1427397600788,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5573,"Index":563,"Attempt":0,"Launch Time":1427397600782,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600789,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":284600,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5577,"Index":567,"Attempt":0,"Launch Time":1427397600792,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5578,"Index":568,"Attempt":0,"Launch Time":1427397600792,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5574,"Index":564,"Attempt":0,"Launch Time":1427397600785,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600792,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":330669,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5570,"Index":560,"Attempt":0,"Launch Time":1427397600771,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600792,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":314314,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5579,"Index":569,"Attempt":0,"Launch Time":1427397600795,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5576,"Index":566,"Attempt":0,"Launch Time":1427397600788,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600795,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":312722,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5580,"Index":570,"Attempt":0,"Launch Time":1427397600796,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5575,"Index":565,"Attempt":0,"Launch Time":1427397600788,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600796,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":308973,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5581,"Index":571,"Attempt":0,"Launch Time":1427397600802,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5579,"Index":569,"Attempt":0,"Launch Time":1427397600795,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600803,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":351744,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5582,"Index":572,"Attempt":0,"Launch Time":1427397600804,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5578,"Index":568,"Attempt":0,"Launch Time":1427397600792,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600804,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":335740,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5583,"Index":573,"Attempt":0,"Launch Time":1427397600804,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5580,"Index":570,"Attempt":0,"Launch Time":1427397600796,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600804,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":348128,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5584,"Index":574,"Attempt":0,"Launch Time":1427397600809,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5581,"Index":571,"Attempt":0,"Launch Time":1427397600802,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600810,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":355612,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5585,"Index":575,"Attempt":0,"Launch Time":1427397600811,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5582,"Index":572,"Attempt":0,"Launch Time":1427397600804,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600811,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9614,"Shuffle Write Time":313182,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5586,"Index":576,"Attempt":0,"Launch Time":1427397600817,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5516,"Index":506,"Attempt":0,"Launch Time":1427397600645,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600817,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":59,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":306739,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5587,"Index":577,"Attempt":0,"Launch Time":1427397600817,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5584,"Index":574,"Attempt":0,"Launch Time":1427397600809,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600817,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":312191,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5588,"Index":578,"Attempt":0,"Launch Time":1427397600818,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5583,"Index":573,"Attempt":0,"Launch Time":1427397600804,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600818,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":2032966,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5589,"Index":579,"Attempt":0,"Launch Time":1427397600821,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5585,"Index":575,"Attempt":0,"Launch Time":1427397600811,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600821,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":278725,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5590,"Index":580,"Attempt":0,"Launch Time":1427397600824,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5587,"Index":577,"Attempt":0,"Launch Time":1427397600817,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600824,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":320803,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5591,"Index":581,"Attempt":0,"Launch Time":1427397600824,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5588,"Index":578,"Attempt":0,"Launch Time":1427397600818,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600824,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":293936,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5592,"Index":582,"Attempt":0,"Launch Time":1427397600825,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5577,"Index":567,"Attempt":0,"Launch Time":1427397600792,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600825,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":319899,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5593,"Index":583,"Attempt":0,"Launch Time":1427397600830,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5590,"Index":580,"Attempt":0,"Launch Time":1427397600824,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600830,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9611,"Shuffle Write Time":314620,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5594,"Index":584,"Attempt":0,"Launch Time":1427397600832,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5589,"Index":579,"Attempt":0,"Launch Time":1427397600821,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600832,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":347934,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5595,"Index":585,"Attempt":0,"Launch Time":1427397600833,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5591,"Index":581,"Attempt":0,"Launch Time":1427397600824,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600833,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":336839,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5596,"Index":586,"Attempt":0,"Launch Time":1427397600837,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5593,"Index":583,"Attempt":0,"Launch Time":1427397600830,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600838,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":348005,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5597,"Index":587,"Attempt":0,"Launch Time":1427397600839,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5594,"Index":584,"Attempt":0,"Launch Time":1427397600832,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600840,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":353228,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5598,"Index":588,"Attempt":0,"Launch Time":1427397600859,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5566,"Index":556,"Attempt":0,"Launch Time":1427397600762,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600860,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":95,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":327089,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5599,"Index":589,"Attempt":0,"Launch Time":1427397600860,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5596,"Index":586,"Attempt":0,"Launch Time":1427397600837,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600860,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":22,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":309888,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5600,"Index":590,"Attempt":0,"Launch Time":1427397600862,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5567,"Index":557,"Attempt":0,"Launch Time":1427397600768,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600862,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":94,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":374043,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5601,"Index":591,"Attempt":0,"Launch Time":1427397600863,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5595,"Index":585,"Attempt":0,"Launch Time":1427397600833,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600863,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":29,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":386742,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5602,"Index":592,"Attempt":0,"Launch Time":1427397600869,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5599,"Index":589,"Attempt":0,"Launch Time":1427397600860,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600870,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":327507,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5603,"Index":593,"Attempt":0,"Launch Time":1427397600870,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5598,"Index":588,"Attempt":0,"Launch Time":1427397600859,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600870,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9608,"Shuffle Write Time":379876,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5604,"Index":594,"Attempt":0,"Launch Time":1427397600871,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5592,"Index":582,"Attempt":0,"Launch Time":1427397600825,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600871,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":32,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":387112,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5605,"Index":595,"Attempt":0,"Launch Time":1427397600877,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5597,"Index":587,"Attempt":0,"Launch Time":1427397600839,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600877,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":429201,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5606,"Index":596,"Attempt":0,"Launch Time":1427397600878,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5607,"Index":597,"Attempt":0,"Launch Time":1427397600878,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5608,"Index":598,"Attempt":0,"Launch Time":1427397600878,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600885,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5601,"Index":591,"Attempt":0,"Launch Time":1427397600863,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600878,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":351806,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5586,"Index":576,"Attempt":0,"Launch Time":1427397600817,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600878,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":19,"Executor Run Time":18,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":334342,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5609,"Index":599,"Attempt":0,"Launch Time":1427397600879,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5604,"Index":594,"Attempt":0,"Launch Time":1427397600871,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600879,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":373265,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5602,"Index":592,"Attempt":0,"Launch Time":1427397600869,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600879,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":286214,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5610,"Index":600,"Attempt":0,"Launch Time":1427397600884,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600893,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5605,"Index":595,"Attempt":0,"Launch Time":1427397600877,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600885,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":317358,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5611,"Index":601,"Attempt":0,"Launch Time":1427397600885,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5606,"Index":596,"Attempt":0,"Launch Time":1427397600878,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600885,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":310521,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5612,"Index":602,"Attempt":0,"Launch Time":1427397600885,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5608,"Index":598,"Attempt":0,"Launch Time":1427397600878,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600885,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":311072,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5613,"Index":603,"Attempt":0,"Launch Time":1427397600892,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5614,"Index":604,"Attempt":0,"Launch Time":1427397600893,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5610,"Index":600,"Attempt":0,"Launch Time":1427397600884,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600893,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":285253,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5607,"Index":597,"Attempt":0,"Launch Time":1427397600878,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600893,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":514505,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5615,"Index":605,"Attempt":0,"Launch Time":1427397600893,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5603,"Index":593,"Attempt":0,"Launch Time":1427397600870,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600893,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":345952,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5616,"Index":606,"Attempt":0,"Launch Time":1427397600899,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5617,"Index":607,"Attempt":0,"Launch Time":1427397600900,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5600,"Index":590,"Attempt":0,"Launch Time":1427397600862,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600900,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":322784,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5612,"Index":602,"Attempt":0,"Launch Time":1427397600885,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600912,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":377697,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5618,"Index":608,"Attempt":0,"Launch Time":1427397600912,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5619,"Index":609,"Attempt":0,"Launch Time":1427397600912,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5615,"Index":605,"Attempt":0,"Launch Time":1427397600893,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600912,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":7,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":324538,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5613,"Index":603,"Attempt":0,"Launch Time":1427397600892,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600912,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":293493,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5617,"Index":607,"Attempt":0,"Launch Time":1427397600900,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600912,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":272877,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5620,"Index":610,"Attempt":0,"Launch Time":1427397600913,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5621,"Index":611,"Attempt":0,"Launch Time":1427397600913,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5611,"Index":601,"Attempt":0,"Launch Time":1427397600885,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600913,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":7,"Executor Run Time":19,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":830964,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5622,"Index":612,"Attempt":0,"Launch Time":1427397600913,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5623,"Index":613,"Attempt":0,"Launch Time":1427397600914,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5616,"Index":606,"Attempt":0,"Launch Time":1427397600899,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600914,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":297815,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5569,"Index":559,"Attempt":0,"Launch Time":1427397600770,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600914,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":25,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":558814,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5624,"Index":614,"Attempt":0,"Launch Time":1427397600922,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5625,"Index":615,"Attempt":0,"Launch Time":1427397600923,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5623,"Index":613,"Attempt":0,"Launch Time":1427397600914,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600923,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":338446,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5621,"Index":611,"Attempt":0,"Launch Time":1427397600913,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600923,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":365329,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5626,"Index":616,"Attempt":0,"Launch Time":1427397600923,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5620,"Index":610,"Attempt":0,"Launch Time":1427397600913,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600923,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":416234,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5627,"Index":617,"Attempt":0,"Launch Time":1427397600926,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5622,"Index":612,"Attempt":0,"Launch Time":1427397600913,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600926,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":373000,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5628,"Index":618,"Attempt":0,"Launch Time":1427397600927,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5618,"Index":608,"Attempt":0,"Launch Time":1427397600912,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600927,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":343695,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5629,"Index":619,"Attempt":0,"Launch Time":1427397600930,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5624,"Index":614,"Attempt":0,"Launch Time":1427397600922,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600930,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":347365,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5630,"Index":620,"Attempt":0,"Launch Time":1427397600930,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5625,"Index":615,"Attempt":0,"Launch Time":1427397600923,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600930,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":606672,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5631,"Index":621,"Attempt":0,"Launch Time":1427397600931,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5626,"Index":616,"Attempt":0,"Launch Time":1427397600923,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600931,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":323586,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5632,"Index":622,"Attempt":0,"Launch Time":1427397600932,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5627,"Index":617,"Attempt":0,"Launch Time":1427397600926,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600932,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":291571,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5633,"Index":623,"Attempt":0,"Launch Time":1427397600934,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5628,"Index":618,"Attempt":0,"Launch Time":1427397600927,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600934,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":317631,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5634,"Index":624,"Attempt":0,"Launch Time":1427397600937,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5630,"Index":620,"Attempt":0,"Launch Time":1427397600930,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600937,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":291357,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5635,"Index":625,"Attempt":0,"Launch Time":1427397600938,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5629,"Index":619,"Attempt":0,"Launch Time":1427397600930,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600939,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":300524,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5636,"Index":626,"Attempt":0,"Launch Time":1427397600940,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5633,"Index":623,"Attempt":0,"Launch Time":1427397600934,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600941,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":333005,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5637,"Index":627,"Attempt":0,"Launch Time":1427397600944,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5634,"Index":624,"Attempt":0,"Launch Time":1427397600937,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600944,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":314862,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5638,"Index":628,"Attempt":0,"Launch Time":1427397600946,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5635,"Index":625,"Attempt":0,"Launch Time":1427397600938,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600946,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":320957,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5639,"Index":629,"Attempt":0,"Launch Time":1427397600947,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5636,"Index":626,"Attempt":0,"Launch Time":1427397600940,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600947,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":310542,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5640,"Index":630,"Attempt":0,"Launch Time":1427397600953,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5637,"Index":627,"Attempt":0,"Launch Time":1427397600944,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600953,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":386992,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5641,"Index":631,"Attempt":0,"Launch Time":1427397600953,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5638,"Index":628,"Attempt":0,"Launch Time":1427397600946,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600954,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":373423,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5642,"Index":632,"Attempt":0,"Launch Time":1427397600961,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5639,"Index":629,"Attempt":0,"Launch Time":1427397600947,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600962,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":398455,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5643,"Index":633,"Attempt":0,"Launch Time":1427397600963,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5609,"Index":599,"Attempt":0,"Launch Time":1427397600879,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600963,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":14,"Executor Run Time":70,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":13193761,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5644,"Index":634,"Attempt":0,"Launch Time":1427397600971,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5641,"Index":631,"Attempt":0,"Launch Time":1427397600953,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600971,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":338669,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5645,"Index":635,"Attempt":0,"Launch Time":1427397600972,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5640,"Index":630,"Attempt":0,"Launch Time":1427397600953,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600972,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":18,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":504964,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5646,"Index":636,"Attempt":0,"Launch Time":1427397600973,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5632,"Index":622,"Attempt":0,"Launch Time":1427397600932,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600973,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":28,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":316622,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5647,"Index":637,"Attempt":0,"Launch Time":1427397600973,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5631,"Index":621,"Attempt":0,"Launch Time":1427397600931,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600973,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":29,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":354021,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5648,"Index":638,"Attempt":0,"Launch Time":1427397600978,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5644,"Index":634,"Attempt":0,"Launch Time":1427397600971,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600979,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":316674,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5649,"Index":639,"Attempt":0,"Launch Time":1427397600980,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5645,"Index":635,"Attempt":0,"Launch Time":1427397600972,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600980,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":337963,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5650,"Index":640,"Attempt":0,"Launch Time":1427397600981,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5619,"Index":609,"Attempt":0,"Launch Time":1427397600912,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600982,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":59,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":331513,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5651,"Index":641,"Attempt":0,"Launch Time":1427397600982,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5646,"Index":636,"Attempt":0,"Launch Time":1427397600973,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600982,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":425806,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5652,"Index":642,"Attempt":0,"Launch Time":1427397600983,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5647,"Index":637,"Attempt":0,"Launch Time":1427397600973,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600983,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1147698,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5653,"Index":643,"Attempt":0,"Launch Time":1427397600985,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5648,"Index":638,"Attempt":0,"Launch Time":1427397600978,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600985,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":309926,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5654,"Index":644,"Attempt":0,"Launch Time":1427397600986,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5649,"Index":639,"Attempt":0,"Launch Time":1427397600980,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600986,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":301553,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5655,"Index":645,"Attempt":0,"Launch Time":1427397600988,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5642,"Index":632,"Attempt":0,"Launch Time":1427397600961,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600988,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":26,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":2147133,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5656,"Index":646,"Attempt":0,"Launch Time":1427397600989,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5651,"Index":641,"Attempt":0,"Launch Time":1427397600982,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600989,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":309613,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5657,"Index":647,"Attempt":0,"Launch Time":1427397600991,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5652,"Index":642,"Attempt":0,"Launch Time":1427397600983,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600991,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":319525,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5658,"Index":648,"Attempt":0,"Launch Time":1427397600993,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5654,"Index":644,"Attempt":0,"Launch Time":1427397600986,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600993,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":327012,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5659,"Index":649,"Attempt":0,"Launch Time":1427397600994,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5653,"Index":643,"Attempt":0,"Launch Time":1427397600985,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600994,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":316000,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5660,"Index":650,"Attempt":0,"Launch Time":1427397600998,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5656,"Index":646,"Attempt":0,"Launch Time":1427397600989,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600998,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":306852,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5661,"Index":651,"Attempt":0,"Launch Time":1427397600998,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5657,"Index":647,"Attempt":0,"Launch Time":1427397600991,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600998,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":319928,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5662,"Index":652,"Attempt":0,"Launch Time":1427397601000,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5658,"Index":648,"Attempt":0,"Launch Time":1427397600993,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601000,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":310972,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5663,"Index":653,"Attempt":0,"Launch Time":1427397601001,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5655,"Index":645,"Attempt":0,"Launch Time":1427397600988,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601001,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":377893,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5664,"Index":654,"Attempt":0,"Launch Time":1427397601002,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5659,"Index":649,"Attempt":0,"Launch Time":1427397600994,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601002,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":423917,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5665,"Index":655,"Attempt":0,"Launch Time":1427397601006,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5661,"Index":651,"Attempt":0,"Launch Time":1427397600998,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601006,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":363293,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5666,"Index":656,"Attempt":0,"Launch Time":1427397601009,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5660,"Index":650,"Attempt":0,"Launch Time":1427397600998,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601009,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":538437,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5667,"Index":657,"Attempt":0,"Launch Time":1427397601010,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5668,"Index":658,"Attempt":0,"Launch Time":1427397601010,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5662,"Index":652,"Attempt":0,"Launch Time":1427397601000,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601010,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":344585,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5663,"Index":653,"Attempt":0,"Launch Time":1427397601001,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601010,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":358777,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5669,"Index":659,"Attempt":0,"Launch Time":1427397601011,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5650,"Index":640,"Attempt":0,"Launch Time":1427397600981,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601011,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":332925,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5670,"Index":660,"Attempt":0,"Launch Time":1427397601012,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5664,"Index":654,"Attempt":0,"Launch Time":1427397601002,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601012,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":315047,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5671,"Index":661,"Attempt":0,"Launch Time":1427397601013,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5665,"Index":655,"Attempt":0,"Launch Time":1427397601006,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601013,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":318657,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5672,"Index":662,"Attempt":0,"Launch Time":1427397601016,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5666,"Index":656,"Attempt":0,"Launch Time":1427397601009,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601016,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":324542,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5673,"Index":663,"Attempt":0,"Launch Time":1427397601018,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5667,"Index":657,"Attempt":0,"Launch Time":1427397601010,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601018,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":530865,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5674,"Index":664,"Attempt":0,"Launch Time":1427397601019,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5670,"Index":660,"Attempt":0,"Launch Time":1427397601012,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601019,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":338171,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5675,"Index":665,"Attempt":0,"Launch Time":1427397601019,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5676,"Index":666,"Attempt":0,"Launch Time":1427397601019,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5668,"Index":658,"Attempt":0,"Launch Time":1427397601010,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601019,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":299614,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5671,"Index":661,"Attempt":0,"Launch Time":1427397601013,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601020,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":299418,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5677,"Index":667,"Attempt":0,"Launch Time":1427397601021,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5614,"Index":604,"Attempt":0,"Launch Time":1427397600893,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601021,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":19,"Executor Run Time":108,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":309135,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5678,"Index":668,"Attempt":0,"Launch Time":1427397601025,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5643,"Index":633,"Attempt":0,"Launch Time":1427397600963,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601026,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":275038,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5679,"Index":669,"Attempt":0,"Launch Time":1427397601031,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5677,"Index":667,"Attempt":0,"Launch Time":1427397601021,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601031,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":325754,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5680,"Index":670,"Attempt":0,"Launch Time":1427397601032,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5678,"Index":668,"Attempt":0,"Launch Time":1427397601025,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601032,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":324233,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5681,"Index":671,"Attempt":0,"Launch Time":1427397601040,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5680,"Index":670,"Attempt":0,"Launch Time":1427397601032,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601040,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":334514,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5682,"Index":672,"Attempt":0,"Launch Time":1427397601040,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5669,"Index":659,"Attempt":0,"Launch Time":1427397601011,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601040,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":28,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":328498,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5683,"Index":673,"Attempt":0,"Launch Time":1427397601042,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5679,"Index":669,"Attempt":0,"Launch Time":1427397601031,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601042,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":338256,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5684,"Index":674,"Attempt":0,"Launch Time":1427397601047,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5682,"Index":672,"Attempt":0,"Launch Time":1427397601040,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601049,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":298502,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5685,"Index":675,"Attempt":0,"Launch Time":1427397601049,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5681,"Index":671,"Attempt":0,"Launch Time":1427397601040,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601049,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":354326,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5686,"Index":676,"Attempt":0,"Launch Time":1427397601050,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5683,"Index":673,"Attempt":0,"Launch Time":1427397601042,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601050,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":463184,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5687,"Index":677,"Attempt":0,"Launch Time":1427397601056,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5684,"Index":674,"Attempt":0,"Launch Time":1427397601047,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601056,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":468696,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5688,"Index":678,"Attempt":0,"Launch Time":1427397601068,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5685,"Index":675,"Attempt":0,"Launch Time":1427397601049,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601068,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":365367,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5689,"Index":679,"Attempt":0,"Launch Time":1427397601075,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5688,"Index":678,"Attempt":0,"Launch Time":1427397601068,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601075,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":316379,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5690,"Index":680,"Attempt":0,"Launch Time":1427397601090,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5689,"Index":679,"Attempt":0,"Launch Time":1427397601075,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601090,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":387118,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5691,"Index":681,"Attempt":0,"Launch Time":1427397601091,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5686,"Index":676,"Attempt":0,"Launch Time":1427397601050,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601091,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":39,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":378631,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5692,"Index":682,"Attempt":0,"Launch Time":1427397601095,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5687,"Index":677,"Attempt":0,"Launch Time":1427397601056,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601095,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":36,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":326610,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5693,"Index":683,"Attempt":0,"Launch Time":1427397601097,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5675,"Index":665,"Attempt":0,"Launch Time":1427397601019,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601097,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":41,"Executor Run Time":8,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":308809,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5694,"Index":684,"Attempt":0,"Launch Time":1427397601098,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5676,"Index":666,"Attempt":0,"Launch Time":1427397601019,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601098,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":749091,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5695,"Index":685,"Attempt":0,"Launch Time":1427397601100,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5691,"Index":681,"Attempt":0,"Launch Time":1427397601091,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601100,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":325165,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5696,"Index":686,"Attempt":0,"Launch Time":1427397601100,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5690,"Index":680,"Attempt":0,"Launch Time":1427397601090,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601100,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":324433,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5697,"Index":687,"Attempt":0,"Launch Time":1427397601101,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5692,"Index":682,"Attempt":0,"Launch Time":1427397601095,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601101,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":307665,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5698,"Index":688,"Attempt":0,"Launch Time":1427397601106,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5694,"Index":684,"Attempt":0,"Launch Time":1427397601098,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601106,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":337131,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5699,"Index":689,"Attempt":0,"Launch Time":1427397601107,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5696,"Index":686,"Attempt":0,"Launch Time":1427397601100,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601107,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":332967,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5700,"Index":690,"Attempt":0,"Launch Time":1427397601108,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5697,"Index":687,"Attempt":0,"Launch Time":1427397601101,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601108,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":328421,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5701,"Index":691,"Attempt":0,"Launch Time":1427397601109,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5674,"Index":664,"Attempt":0,"Launch Time":1427397601019,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601109,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":50,"Executor Run Time":33,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":358222,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5702,"Index":692,"Attempt":0,"Launch Time":1427397601110,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5693,"Index":683,"Attempt":0,"Launch Time":1427397601097,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601110,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":319844,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5703,"Index":693,"Attempt":0,"Launch Time":1427397601114,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5698,"Index":688,"Attempt":0,"Launch Time":1427397601106,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601114,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":395414,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5704,"Index":694,"Attempt":0,"Launch Time":1427397601115,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5699,"Index":689,"Attempt":0,"Launch Time":1427397601107,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601115,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":294895,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5705,"Index":695,"Attempt":0,"Launch Time":1427397601115,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5673,"Index":663,"Attempt":0,"Launch Time":1427397601018,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601116,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":8,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":312668,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5706,"Index":696,"Attempt":0,"Launch Time":1427397601116,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5700,"Index":690,"Attempt":0,"Launch Time":1427397601108,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601116,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":314132,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5707,"Index":697,"Attempt":0,"Launch Time":1427397601117,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5695,"Index":685,"Attempt":0,"Launch Time":1427397601100,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601117,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":536305,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5708,"Index":698,"Attempt":0,"Launch Time":1427397601118,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5702,"Index":692,"Attempt":0,"Launch Time":1427397601110,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601118,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":368248,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5709,"Index":699,"Attempt":0,"Launch Time":1427397601123,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5705,"Index":695,"Attempt":0,"Launch Time":1427397601115,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601123,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":338077,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5710,"Index":700,"Attempt":0,"Launch Time":1427397601123,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5706,"Index":696,"Attempt":0,"Launch Time":1427397601116,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601123,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":349851,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5711,"Index":701,"Attempt":0,"Launch Time":1427397601124,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5704,"Index":694,"Attempt":0,"Launch Time":1427397601115,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601124,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":349835,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5712,"Index":702,"Attempt":0,"Launch Time":1427397601128,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5708,"Index":698,"Attempt":0,"Launch Time":1427397601118,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601128,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":320898,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5713,"Index":703,"Attempt":0,"Launch Time":1427397601130,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5709,"Index":699,"Attempt":0,"Launch Time":1427397601123,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601130,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":364914,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5714,"Index":704,"Attempt":0,"Launch Time":1427397601132,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5711,"Index":701,"Attempt":0,"Launch Time":1427397601124,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601132,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":347125,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5715,"Index":705,"Attempt":0,"Launch Time":1427397601134,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5707,"Index":697,"Attempt":0,"Launch Time":1427397601117,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601134,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9612,"Shuffle Write Time":285440,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5716,"Index":706,"Attempt":0,"Launch Time":1427397601137,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5713,"Index":703,"Attempt":0,"Launch Time":1427397601130,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601137,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":299467,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5717,"Index":707,"Attempt":0,"Launch Time":1427397601139,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5714,"Index":704,"Attempt":0,"Launch Time":1427397601132,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601139,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":318096,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5718,"Index":708,"Attempt":0,"Launch Time":1427397601140,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5712,"Index":702,"Attempt":0,"Launch Time":1427397601128,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601140,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":323709,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5719,"Index":709,"Attempt":0,"Launch Time":1427397601141,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5710,"Index":700,"Attempt":0,"Launch Time":1427397601123,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601141,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":282744,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5720,"Index":710,"Attempt":0,"Launch Time":1427397601146,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5717,"Index":707,"Attempt":0,"Launch Time":1427397601139,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601146,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":362438,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5721,"Index":711,"Attempt":0,"Launch Time":1427397601146,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5722,"Index":712,"Attempt":0,"Launch Time":1427397601146,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5716,"Index":706,"Attempt":0,"Launch Time":1427397601137,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601147,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":364709,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5715,"Index":705,"Attempt":0,"Launch Time":1427397601134,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601147,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9615,"Shuffle Write Time":403208,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5723,"Index":713,"Attempt":0,"Launch Time":1427397601147,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5718,"Index":708,"Attempt":0,"Launch Time":1427397601140,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601147,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":378301,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5724,"Index":714,"Attempt":0,"Launch Time":1427397601148,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5701,"Index":691,"Attempt":0,"Launch Time":1427397601109,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601148,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":6,"Executor Run Time":31,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":300864,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5725,"Index":715,"Attempt":0,"Launch Time":1427397601148,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5672,"Index":662,"Attempt":0,"Launch Time":1427397601016,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601148,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":131,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":387955,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5726,"Index":716,"Attempt":0,"Launch Time":1427397601154,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5703,"Index":693,"Attempt":0,"Launch Time":1427397601114,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601154,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":37,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":339508,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5727,"Index":717,"Attempt":0,"Launch Time":1427397601157,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5725,"Index":715,"Attempt":0,"Launch Time":1427397601148,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601157,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":387737,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5728,"Index":718,"Attempt":0,"Launch Time":1427397601160,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5719,"Index":709,"Attempt":0,"Launch Time":1427397601141,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601160,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":401215,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5729,"Index":719,"Attempt":0,"Launch Time":1427397601163,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5730,"Index":720,"Attempt":0,"Launch Time":1427397601163,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5726,"Index":716,"Attempt":0,"Launch Time":1427397601154,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601163,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":305694,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5724,"Index":714,"Attempt":0,"Launch Time":1427397601148,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601164,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":553085,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5731,"Index":721,"Attempt":0,"Launch Time":1427397601164,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5727,"Index":717,"Attempt":0,"Launch Time":1427397601157,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601164,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":314562,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5732,"Index":722,"Attempt":0,"Launch Time":1427397601167,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5728,"Index":718,"Attempt":0,"Launch Time":1427397601160,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601167,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":314281,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5733,"Index":723,"Attempt":0,"Launch Time":1427397601168,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5723,"Index":713,"Attempt":0,"Launch Time":1427397601147,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601168,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9613,"Shuffle Write Time":316394,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5734,"Index":724,"Attempt":0,"Launch Time":1427397601181,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5730,"Index":720,"Attempt":0,"Launch Time":1427397601163,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601181,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":347976,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5735,"Index":725,"Attempt":0,"Launch Time":1427397601196,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5729,"Index":719,"Attempt":0,"Launch Time":1427397601163,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601196,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":29,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":23586631,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5736,"Index":726,"Attempt":0,"Launch Time":1427397601196,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5731,"Index":721,"Attempt":0,"Launch Time":1427397601164,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601196,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":29,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":300287,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5737,"Index":727,"Attempt":0,"Launch Time":1427397601199,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5732,"Index":722,"Attempt":0,"Launch Time":1427397601167,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601199,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":31,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":356003,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5738,"Index":728,"Attempt":0,"Launch Time":1427397601207,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5737,"Index":727,"Attempt":0,"Launch Time":1427397601199,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601208,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":359402,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5739,"Index":729,"Attempt":0,"Launch Time":1427397601209,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5736,"Index":726,"Attempt":0,"Launch Time":1427397601196,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601209,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":499241,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5740,"Index":730,"Attempt":0,"Launch Time":1427397601209,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5735,"Index":725,"Attempt":0,"Launch Time":1427397601196,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601209,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":920670,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5741,"Index":731,"Attempt":0,"Launch Time":1427397601210,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5734,"Index":724,"Attempt":0,"Launch Time":1427397601181,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601210,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":28,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1412322,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5742,"Index":732,"Attempt":0,"Launch Time":1427397601214,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5743,"Index":733,"Attempt":0,"Launch Time":1427397601214,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5733,"Index":723,"Attempt":0,"Launch Time":1427397601168,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601215,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":523902,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5738,"Index":728,"Attempt":0,"Launch Time":1427397601207,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601215,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":275163,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5744,"Index":734,"Attempt":0,"Launch Time":1427397601215,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5739,"Index":729,"Attempt":0,"Launch Time":1427397601209,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601215,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":301051,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5745,"Index":735,"Attempt":0,"Launch Time":1427397601215,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5740,"Index":730,"Attempt":0,"Launch Time":1427397601209,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601215,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":301771,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5746,"Index":736,"Attempt":0,"Launch Time":1427397601217,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5721,"Index":711,"Attempt":0,"Launch Time":1427397601146,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601218,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":22,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":318947,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5747,"Index":737,"Attempt":0,"Launch Time":1427397601220,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5741,"Index":731,"Attempt":0,"Launch Time":1427397601210,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601220,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":556661,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5748,"Index":738,"Attempt":0,"Launch Time":1427397601221,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5744,"Index":734,"Attempt":0,"Launch Time":1427397601215,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601222,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":310357,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5749,"Index":739,"Attempt":0,"Launch Time":1427397601222,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5745,"Index":735,"Attempt":0,"Launch Time":1427397601215,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601222,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":365423,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5750,"Index":740,"Attempt":0,"Launch Time":1427397601223,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5720,"Index":710,"Attempt":0,"Launch Time":1427397601146,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601223,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":7,"Executor Run Time":57,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1258151,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5751,"Index":741,"Attempt":0,"Launch Time":1427397601224,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5743,"Index":733,"Attempt":0,"Launch Time":1427397601214,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601224,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":303927,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5742,"Index":732,"Attempt":0,"Launch Time":1427397601214,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601224,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":327533,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5752,"Index":742,"Attempt":0,"Launch Time":1427397601224,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5753,"Index":743,"Attempt":0,"Launch Time":1427397601230,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5749,"Index":739,"Attempt":0,"Launch Time":1427397601222,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601230,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":301566,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5754,"Index":744,"Attempt":0,"Launch Time":1427397601231,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5752,"Index":742,"Attempt":0,"Launch Time":1427397601224,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601231,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":305516,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5755,"Index":745,"Attempt":0,"Launch Time":1427397601231,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5747,"Index":737,"Attempt":0,"Launch Time":1427397601220,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601231,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":312951,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5756,"Index":746,"Attempt":0,"Launch Time":1427397601233,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5748,"Index":738,"Attempt":0,"Launch Time":1427397601221,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601234,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1031507,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5757,"Index":747,"Attempt":0,"Launch Time":1427397601238,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5751,"Index":741,"Attempt":0,"Launch Time":1427397601224,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601238,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":293037,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5758,"Index":748,"Attempt":0,"Launch Time":1427397601239,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5753,"Index":743,"Attempt":0,"Launch Time":1427397601230,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601240,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":325769,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5759,"Index":749,"Attempt":0,"Launch Time":1427397601243,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5755,"Index":745,"Attempt":0,"Launch Time":1427397601231,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601243,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":316122,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5760,"Index":750,"Attempt":0,"Launch Time":1427397601244,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5754,"Index":744,"Attempt":0,"Launch Time":1427397601231,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601244,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":358312,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5761,"Index":751,"Attempt":0,"Launch Time":1427397601246,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5756,"Index":746,"Attempt":0,"Launch Time":1427397601233,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601246,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":321368,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5762,"Index":752,"Attempt":0,"Launch Time":1427397601248,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5758,"Index":748,"Attempt":0,"Launch Time":1427397601239,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601248,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1079928,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5763,"Index":753,"Attempt":0,"Launch Time":1427397601250,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5757,"Index":747,"Attempt":0,"Launch Time":1427397601238,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601250,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1580937,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5764,"Index":754,"Attempt":0,"Launch Time":1427397601251,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5760,"Index":750,"Attempt":0,"Launch Time":1427397601244,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601251,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":366931,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5765,"Index":755,"Attempt":0,"Launch Time":1427397601255,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5759,"Index":749,"Attempt":0,"Launch Time":1427397601243,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601256,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":398845,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5766,"Index":756,"Attempt":0,"Launch Time":1427397601256,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5761,"Index":751,"Attempt":0,"Launch Time":1427397601246,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601257,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":754150,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5767,"Index":757,"Attempt":0,"Launch Time":1427397601258,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5768,"Index":758,"Attempt":0,"Launch Time":1427397601258,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5763,"Index":753,"Attempt":0,"Launch Time":1427397601250,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601258,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":368832,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5762,"Index":752,"Attempt":0,"Launch Time":1427397601248,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601258,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":439831,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5769,"Index":759,"Attempt":0,"Launch Time":1427397601258,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5764,"Index":754,"Attempt":0,"Launch Time":1427397601251,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601258,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":341142,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5770,"Index":760,"Attempt":0,"Launch Time":1427397601259,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5750,"Index":740,"Attempt":0,"Launch Time":1427397601223,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601259,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":34,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":3856503,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5771,"Index":761,"Attempt":0,"Launch Time":1427397601266,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5772,"Index":762,"Attempt":0,"Launch Time":1427397601266,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5773,"Index":763,"Attempt":0,"Launch Time":1427397601266,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5774,"Index":764,"Attempt":0,"Launch Time":1427397601266,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5765,"Index":755,"Attempt":0,"Launch Time":1427397601255,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601266,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":320071,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5766,"Index":756,"Attempt":0,"Launch Time":1427397601256,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601266,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":329400,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5768,"Index":758,"Attempt":0,"Launch Time":1427397601258,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601267,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":310537,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5769,"Index":759,"Attempt":0,"Launch Time":1427397601258,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601267,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":289026,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5775,"Index":765,"Attempt":0,"Launch Time":1427397601269,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5767,"Index":757,"Attempt":0,"Launch Time":1427397601258,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601269,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":295198,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5776,"Index":766,"Attempt":0,"Launch Time":1427397601272,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5722,"Index":712,"Attempt":0,"Launch Time":1427397601146,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601274,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":81,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":345394,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5777,"Index":767,"Attempt":0,"Launch Time":1427397601274,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5778,"Index":768,"Attempt":0,"Launch Time":1427397601274,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5770,"Index":760,"Attempt":0,"Launch Time":1427397601259,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601275,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":5,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":321004,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5746,"Index":736,"Attempt":0,"Launch Time":1427397601217,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601275,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":56,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":315876,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5779,"Index":769,"Attempt":0,"Launch Time":1427397601290,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5776,"Index":766,"Attempt":0,"Launch Time":1427397601272,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601290,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":317284,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5780,"Index":770,"Attempt":0,"Launch Time":1427397601300,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5772,"Index":762,"Attempt":0,"Launch Time":1427397601266,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601300,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":31,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":406392,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5781,"Index":771,"Attempt":0,"Launch Time":1427397601300,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5779,"Index":769,"Attempt":0,"Launch Time":1427397601290,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601301,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":342522,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5782,"Index":772,"Attempt":0,"Launch Time":1427397601310,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5783,"Index":773,"Attempt":0,"Launch Time":1427397601310,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5773,"Index":763,"Attempt":0,"Launch Time":1427397601266,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601310,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":38,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":325521,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5778,"Index":768,"Attempt":0,"Launch Time":1427397601274,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601310,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":31,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":370232,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5784,"Index":774,"Attempt":0,"Launch Time":1427397601310,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5780,"Index":770,"Attempt":0,"Launch Time":1427397601300,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601310,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":340782,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5785,"Index":775,"Attempt":0,"Launch Time":1427397601310,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5777,"Index":767,"Attempt":0,"Launch Time":1427397601274,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601310,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":317306,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5786,"Index":776,"Attempt":0,"Launch Time":1427397601313,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5781,"Index":771,"Attempt":0,"Launch Time":1427397601300,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601313,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":329765,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5787,"Index":777,"Attempt":0,"Launch Time":1427397601313,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5771,"Index":761,"Attempt":0,"Launch Time":1427397601266,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601313,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":46,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":488134,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5788,"Index":778,"Attempt":0,"Launch Time":1427397601316,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5775,"Index":765,"Attempt":0,"Launch Time":1427397601269,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601316,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1743013,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5789,"Index":779,"Attempt":0,"Launch Time":1427397601317,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5790,"Index":780,"Attempt":0,"Launch Time":1427397601318,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5785,"Index":775,"Attempt":0,"Launch Time":1427397601310,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601318,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":321260,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5783,"Index":773,"Attempt":0,"Launch Time":1427397601310,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601318,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":336804,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5791,"Index":781,"Attempt":0,"Launch Time":1427397601319,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5784,"Index":774,"Attempt":0,"Launch Time":1427397601310,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601319,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":335211,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5792,"Index":782,"Attempt":0,"Launch Time":1427397601321,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5774,"Index":764,"Attempt":0,"Launch Time":1427397601266,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601322,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":54,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1091896,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5793,"Index":783,"Attempt":0,"Launch Time":1427397601326,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5790,"Index":780,"Attempt":0,"Launch Time":1427397601318,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601327,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":317926,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5794,"Index":784,"Attempt":0,"Launch Time":1427397601327,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5791,"Index":781,"Attempt":0,"Launch Time":1427397601319,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601327,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":320466,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5795,"Index":785,"Attempt":0,"Launch Time":1427397601334,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5789,"Index":779,"Attempt":0,"Launch Time":1427397601317,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601334,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":710871,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5796,"Index":786,"Attempt":0,"Launch Time":1427397601335,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5794,"Index":784,"Attempt":0,"Launch Time":1427397601327,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601335,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":343508,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5797,"Index":787,"Attempt":0,"Launch Time":1427397601336,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5793,"Index":783,"Attempt":0,"Launch Time":1427397601326,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601337,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1347933,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5798,"Index":788,"Attempt":0,"Launch Time":1427397601338,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5788,"Index":778,"Attempt":0,"Launch Time":1427397601316,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601338,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":2153296,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5799,"Index":789,"Attempt":0,"Launch Time":1427397601342,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5800,"Index":790,"Attempt":0,"Launch Time":1427397601343,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5795,"Index":785,"Attempt":0,"Launch Time":1427397601334,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601343,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":478773,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5796,"Index":786,"Attempt":0,"Launch Time":1427397601335,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601343,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":324285,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5801,"Index":791,"Attempt":0,"Launch Time":1427397601344,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5797,"Index":787,"Attempt":0,"Launch Time":1427397601336,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601344,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":336545,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5802,"Index":792,"Attempt":0,"Launch Time":1427397601346,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5786,"Index":776,"Attempt":0,"Launch Time":1427397601313,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601346,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":29,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":432144,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5803,"Index":793,"Attempt":0,"Launch Time":1427397601349,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5798,"Index":788,"Attempt":0,"Launch Time":1427397601338,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601349,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":580656,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5804,"Index":794,"Attempt":0,"Launch Time":1427397601351,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5801,"Index":791,"Attempt":0,"Launch Time":1427397601344,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601351,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":362029,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5805,"Index":795,"Attempt":0,"Launch Time":1427397601352,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5792,"Index":782,"Attempt":0,"Launch Time":1427397601321,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601352,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":343516,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5806,"Index":796,"Attempt":0,"Launch Time":1427397601353,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5782,"Index":772,"Attempt":0,"Launch Time":1427397601310,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601353,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":23,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":349339,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5807,"Index":797,"Attempt":0,"Launch Time":1427397601355,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5800,"Index":790,"Attempt":0,"Launch Time":1427397601343,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601356,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":352394,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5808,"Index":798,"Attempt":0,"Launch Time":1427397601358,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5804,"Index":794,"Attempt":0,"Launch Time":1427397601351,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601358,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":367453,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5809,"Index":799,"Attempt":0,"Launch Time":1427397601359,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5799,"Index":789,"Attempt":0,"Launch Time":1427397601342,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601359,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":589876,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5810,"Index":800,"Attempt":0,"Launch Time":1427397601361,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5802,"Index":792,"Attempt":0,"Launch Time":1427397601346,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601361,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":291736,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5811,"Index":801,"Attempt":0,"Launch Time":1427397601362,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5807,"Index":797,"Attempt":0,"Launch Time":1427397601355,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601362,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":275020,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5812,"Index":802,"Attempt":0,"Launch Time":1427397601364,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5803,"Index":793,"Attempt":0,"Launch Time":1427397601349,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601364,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":520528,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5813,"Index":803,"Attempt":0,"Launch Time":1427397601365,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5808,"Index":798,"Attempt":0,"Launch Time":1427397601358,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601365,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":307010,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5814,"Index":804,"Attempt":0,"Launch Time":1427397601366,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5809,"Index":799,"Attempt":0,"Launch Time":1427397601359,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601369,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":342688,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5815,"Index":805,"Attempt":0,"Launch Time":1427397601369,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5811,"Index":801,"Attempt":0,"Launch Time":1427397601362,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601370,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":283342,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5816,"Index":806,"Attempt":0,"Launch Time":1427397601371,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5813,"Index":803,"Attempt":0,"Launch Time":1427397601365,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601371,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":278805,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5817,"Index":807,"Attempt":0,"Launch Time":1427397601371,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5812,"Index":802,"Attempt":0,"Launch Time":1427397601364,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601371,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":287047,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5818,"Index":808,"Attempt":0,"Launch Time":1427397601377,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5815,"Index":805,"Attempt":0,"Launch Time":1427397601369,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601377,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":306922,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5819,"Index":809,"Attempt":0,"Launch Time":1427397601378,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5814,"Index":804,"Attempt":0,"Launch Time":1427397601366,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601378,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":327891,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5820,"Index":810,"Attempt":0,"Launch Time":1427397601379,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5817,"Index":807,"Attempt":0,"Launch Time":1427397601371,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601379,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":312922,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5821,"Index":811,"Attempt":0,"Launch Time":1427397601385,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5822,"Index":812,"Attempt":0,"Launch Time":1427397601385,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5819,"Index":809,"Attempt":0,"Launch Time":1427397601378,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601385,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":335487,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5818,"Index":808,"Attempt":0,"Launch Time":1427397601377,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601385,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":323511,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5823,"Index":813,"Attempt":0,"Launch Time":1427397601386,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5820,"Index":810,"Attempt":0,"Launch Time":1427397601379,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601386,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":334259,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5824,"Index":814,"Attempt":0,"Launch Time":1427397601436,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5787,"Index":777,"Attempt":0,"Launch Time":1427397601313,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601436,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":57,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":424299,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5825,"Index":815,"Attempt":0,"Launch Time":1427397601437,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5805,"Index":795,"Attempt":0,"Launch Time":1427397601352,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601437,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":38,"Executor Run Time":7,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":328797,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5826,"Index":816,"Attempt":0,"Launch Time":1427397601437,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5806,"Index":796,"Attempt":0,"Launch Time":1427397601353,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601437,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":328248,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5816,"Index":806,"Attempt":0,"Launch Time":1427397601371,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601438,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":31,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":323084,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5827,"Index":817,"Attempt":0,"Launch Time":1427397601438,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5828,"Index":818,"Attempt":0,"Launch Time":1427397601438,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5829,"Index":819,"Attempt":0,"Launch Time":1427397601438,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5830,"Index":820,"Attempt":0,"Launch Time":1427397601440,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5810,"Index":800,"Attempt":0,"Launch Time":1427397601361,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601441,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":42,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":22269690,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5821,"Index":811,"Attempt":0,"Launch Time":1427397601385,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601441,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":30,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":305665,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5822,"Index":812,"Attempt":0,"Launch Time":1427397601385,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601441,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":27,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":338356,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5831,"Index":821,"Attempt":0,"Launch Time":1427397601442,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5823,"Index":813,"Attempt":0,"Launch Time":1427397601386,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601442,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":38,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":333221,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5832,"Index":822,"Attempt":0,"Launch Time":1427397601443,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5825,"Index":815,"Attempt":0,"Launch Time":1427397601437,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601443,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":291100,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5833,"Index":823,"Attempt":0,"Launch Time":1427397601446,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5828,"Index":818,"Attempt":0,"Launch Time":1427397601438,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601446,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":291997,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5834,"Index":824,"Attempt":0,"Launch Time":1427397601448,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5831,"Index":821,"Attempt":0,"Launch Time":1427397601442,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601449,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":281603,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5835,"Index":825,"Attempt":0,"Launch Time":1427397601449,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5832,"Index":822,"Attempt":0,"Launch Time":1427397601443,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601449,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":300053,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5836,"Index":826,"Attempt":0,"Launch Time":1427397601454,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5833,"Index":823,"Attempt":0,"Launch Time":1427397601446,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601454,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":337180,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5837,"Index":827,"Attempt":0,"Launch Time":1427397601456,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5834,"Index":824,"Attempt":0,"Launch Time":1427397601448,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601456,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":346668,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5838,"Index":828,"Attempt":0,"Launch Time":1427397601461,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5830,"Index":820,"Attempt":0,"Launch Time":1427397601440,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601461,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":318690,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5839,"Index":829,"Attempt":0,"Launch Time":1427397601463,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5829,"Index":819,"Attempt":0,"Launch Time":1427397601438,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601463,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":562135,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5840,"Index":830,"Attempt":0,"Launch Time":1427397601463,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5836,"Index":826,"Attempt":0,"Launch Time":1427397601454,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601464,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":319985,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5841,"Index":831,"Attempt":0,"Launch Time":1427397601466,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5827,"Index":817,"Attempt":0,"Launch Time":1427397601438,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601466,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":337756,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5842,"Index":832,"Attempt":0,"Launch Time":1427397601470,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5838,"Index":828,"Attempt":0,"Launch Time":1427397601461,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601470,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":316605,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5843,"Index":833,"Attempt":0,"Launch Time":1427397601470,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5826,"Index":816,"Attempt":0,"Launch Time":1427397601437,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601470,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":325449,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5844,"Index":834,"Attempt":0,"Launch Time":1427397601471,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5840,"Index":830,"Attempt":0,"Launch Time":1427397601463,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601471,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":309718,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5845,"Index":835,"Attempt":0,"Launch Time":1427397601474,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5839,"Index":829,"Attempt":0,"Launch Time":1427397601463,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601475,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":361345,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5846,"Index":836,"Attempt":0,"Launch Time":1427397601475,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5841,"Index":831,"Attempt":0,"Launch Time":1427397601466,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601475,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":376527,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5847,"Index":837,"Attempt":0,"Launch Time":1427397601478,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5844,"Index":834,"Attempt":0,"Launch Time":1427397601471,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601478,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":313873,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5848,"Index":838,"Attempt":0,"Launch Time":1427397601479,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5842,"Index":832,"Attempt":0,"Launch Time":1427397601470,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601479,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":362593,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5849,"Index":839,"Attempt":0,"Launch Time":1427397601481,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5843,"Index":833,"Attempt":0,"Launch Time":1427397601470,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601481,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":309066,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5847,"Index":837,"Attempt":0,"Launch Time":1427397601478,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601484,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":264853,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5850,"Index":840,"Attempt":0,"Launch Time":1427397601484,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5851,"Index":841,"Attempt":0,"Launch Time":1427397601485,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5845,"Index":835,"Attempt":0,"Launch Time":1427397601474,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601485,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":386081,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5852,"Index":842,"Attempt":0,"Launch Time":1427397601491,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5848,"Index":838,"Attempt":0,"Launch Time":1427397601479,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601491,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":333862,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5853,"Index":843,"Attempt":0,"Launch Time":1427397601492,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5851,"Index":841,"Attempt":0,"Launch Time":1427397601485,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601492,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":357682,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5854,"Index":844,"Attempt":0,"Launch Time":1427397601492,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5850,"Index":840,"Attempt":0,"Launch Time":1427397601484,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601492,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":387018,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5855,"Index":845,"Attempt":0,"Launch Time":1427397601494,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5849,"Index":839,"Attempt":0,"Launch Time":1427397601481,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601494,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":5,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":413960,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5856,"Index":846,"Attempt":0,"Launch Time":1427397601499,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5852,"Index":842,"Attempt":0,"Launch Time":1427397601491,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601499,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":349457,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5857,"Index":847,"Attempt":0,"Launch Time":1427397601501,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5858,"Index":848,"Attempt":0,"Launch Time":1427397601501,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5855,"Index":845,"Attempt":0,"Launch Time":1427397601494,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601501,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":345943,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5853,"Index":843,"Attempt":0,"Launch Time":1427397601492,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601502,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":331276,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5859,"Index":849,"Attempt":0,"Launch Time":1427397601502,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5860,"Index":850,"Attempt":0,"Launch Time":1427397601502,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5854,"Index":844,"Attempt":0,"Launch Time":1427397601492,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601502,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":350834,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5846,"Index":836,"Attempt":0,"Launch Time":1427397601475,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601502,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":322731,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5856,"Index":846,"Attempt":0,"Launch Time":1427397601499,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601508,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":404911,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5861,"Index":851,"Attempt":0,"Launch Time":1427397601509,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5862,"Index":852,"Attempt":0,"Launch Time":1427397601509,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5857,"Index":847,"Attempt":0,"Launch Time":1427397601501,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601510,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":305156,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5863,"Index":853,"Attempt":0,"Launch Time":1427397601511,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5835,"Index":825,"Attempt":0,"Launch Time":1427397601449,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601511,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":62,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":54537507,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5864,"Index":854,"Attempt":0,"Launch Time":1427397601511,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5858,"Index":848,"Attempt":0,"Launch Time":1427397601501,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601512,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":363062,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5865,"Index":855,"Attempt":0,"Launch Time":1427397601514,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5860,"Index":850,"Attempt":0,"Launch Time":1427397601502,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601514,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":451953,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5866,"Index":856,"Attempt":0,"Launch Time":1427397601517,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5861,"Index":851,"Attempt":0,"Launch Time":1427397601509,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601518,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":302267,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5867,"Index":857,"Attempt":0,"Launch Time":1427397601530,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5866,"Index":856,"Attempt":0,"Launch Time":1427397601517,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601530,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":313749,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5868,"Index":858,"Attempt":0,"Launch Time":1427397601531,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5837,"Index":827,"Attempt":0,"Launch Time":1427397601456,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601531,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":75,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":406011,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5869,"Index":859,"Attempt":0,"Launch Time":1427397601536,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5867,"Index":857,"Attempt":0,"Launch Time":1427397601530,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601536,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":349223,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5870,"Index":860,"Attempt":0,"Launch Time":1427397601538,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5871,"Index":861,"Attempt":0,"Launch Time":1427397601538,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5824,"Index":814,"Attempt":0,"Launch Time":1427397601436,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601538,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":400443,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5868,"Index":858,"Attempt":0,"Launch Time":1427397601531,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601539,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":333966,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5872,"Index":862,"Attempt":0,"Launch Time":1427397601540,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5863,"Index":853,"Attempt":0,"Launch Time":1427397601511,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601540,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":22,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":1276417,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5873,"Index":863,"Attempt":0,"Launch Time":1427397601543,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5869,"Index":859,"Attempt":0,"Launch Time":1427397601536,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601543,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":303941,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5874,"Index":864,"Attempt":0,"Launch Time":1427397601545,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5870,"Index":860,"Attempt":0,"Launch Time":1427397601538,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601545,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":278998,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5875,"Index":865,"Attempt":0,"Launch Time":1427397601545,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5871,"Index":861,"Attempt":0,"Launch Time":1427397601538,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601545,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":313675,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5876,"Index":866,"Attempt":0,"Launch Time":1427397601546,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5872,"Index":862,"Attempt":0,"Launch Time":1427397601540,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601546,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":304444,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5877,"Index":867,"Attempt":0,"Launch Time":1427397601549,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5873,"Index":863,"Attempt":0,"Launch Time":1427397601543,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601549,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":315082,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5878,"Index":868,"Attempt":0,"Launch Time":1427397601552,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5874,"Index":864,"Attempt":0,"Launch Time":1427397601545,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601552,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":376826,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5879,"Index":869,"Attempt":0,"Launch Time":1427397601553,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5875,"Index":865,"Attempt":0,"Launch Time":1427397601545,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601553,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":374981,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5880,"Index":870,"Attempt":0,"Launch Time":1427397601557,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5877,"Index":867,"Attempt":0,"Launch Time":1427397601549,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601558,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":445353,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5881,"Index":871,"Attempt":0,"Launch Time":1427397601559,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5876,"Index":866,"Attempt":0,"Launch Time":1427397601546,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601559,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":383645,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5882,"Index":872,"Attempt":0,"Launch Time":1427397601560,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5883,"Index":873,"Attempt":0,"Launch Time":1427397601560,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5878,"Index":868,"Attempt":0,"Launch Time":1427397601552,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601560,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":310859,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5879,"Index":869,"Attempt":0,"Launch Time":1427397601553,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601560,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":319117,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5884,"Index":874,"Attempt":0,"Launch Time":1427397601564,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5880,"Index":870,"Attempt":0,"Launch Time":1427397601557,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601564,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":271710,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5885,"Index":875,"Attempt":0,"Launch Time":1427397601565,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5881,"Index":871,"Attempt":0,"Launch Time":1427397601559,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601565,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":326337,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5886,"Index":876,"Attempt":0,"Launch Time":1427397601566,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5882,"Index":872,"Attempt":0,"Launch Time":1427397601560,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601566,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":301949,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5887,"Index":877,"Attempt":0,"Launch Time":1427397601566,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5883,"Index":873,"Attempt":0,"Launch Time":1427397601560,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601566,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":283259,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5888,"Index":878,"Attempt":0,"Launch Time":1427397601568,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5864,"Index":854,"Attempt":0,"Launch Time":1427397601511,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601569,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":56,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":360252,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5889,"Index":879,"Attempt":0,"Launch Time":1427397601570,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5884,"Index":874,"Attempt":0,"Launch Time":1427397601564,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601570,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":293958,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5890,"Index":880,"Attempt":0,"Launch Time":1427397601572,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5885,"Index":875,"Attempt":0,"Launch Time":1427397601565,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601572,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":320393,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5891,"Index":881,"Attempt":0,"Launch Time":1427397601573,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5886,"Index":876,"Attempt":0,"Launch Time":1427397601566,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601573,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":294694,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5892,"Index":882,"Attempt":0,"Launch Time":1427397601573,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5887,"Index":877,"Attempt":0,"Launch Time":1427397601566,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601573,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":371283,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5893,"Index":883,"Attempt":0,"Launch Time":1427397601577,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5889,"Index":879,"Attempt":0,"Launch Time":1427397601570,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601577,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":353478,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5894,"Index":884,"Attempt":0,"Launch Time":1427397601579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5890,"Index":880,"Attempt":0,"Launch Time":1427397601572,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601580,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":327953,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5895,"Index":885,"Attempt":0,"Launch Time":1427397601580,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5892,"Index":882,"Attempt":0,"Launch Time":1427397601573,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601580,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":321587,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5896,"Index":886,"Attempt":0,"Launch Time":1427397601581,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5891,"Index":881,"Attempt":0,"Launch Time":1427397601573,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601582,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":320780,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5897,"Index":887,"Attempt":0,"Launch Time":1427397601584,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5893,"Index":883,"Attempt":0,"Launch Time":1427397601577,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601584,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":328612,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5898,"Index":888,"Attempt":0,"Launch Time":1427397601587,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5859,"Index":849,"Attempt":0,"Launch Time":1427397601502,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601587,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":450832,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5899,"Index":889,"Attempt":0,"Launch Time":1427397601587,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5895,"Index":885,"Attempt":0,"Launch Time":1427397601580,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601587,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":322909,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5900,"Index":890,"Attempt":0,"Launch Time":1427397601588,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5894,"Index":884,"Attempt":0,"Launch Time":1427397601579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601588,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":345484,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5901,"Index":891,"Attempt":0,"Launch Time":1427397601589,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5896,"Index":886,"Attempt":0,"Launch Time":1427397601581,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601589,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":314895,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5902,"Index":892,"Attempt":0,"Launch Time":1427397601591,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5897,"Index":887,"Attempt":0,"Launch Time":1427397601584,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601591,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":367026,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5903,"Index":893,"Attempt":0,"Launch Time":1427397601595,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5899,"Index":889,"Attempt":0,"Launch Time":1427397601587,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601595,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":316207,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5904,"Index":894,"Attempt":0,"Launch Time":1427397601595,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5898,"Index":888,"Attempt":0,"Launch Time":1427397601587,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601595,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":323312,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5905,"Index":895,"Attempt":0,"Launch Time":1427397601596,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5900,"Index":890,"Attempt":0,"Launch Time":1427397601588,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601596,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":324353,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5902,"Index":892,"Attempt":0,"Launch Time":1427397601591,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601597,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":294203,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5906,"Index":896,"Attempt":0,"Launch Time":1427397601597,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5907,"Index":897,"Attempt":0,"Launch Time":1427397601599,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5901,"Index":891,"Attempt":0,"Launch Time":1427397601589,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601600,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":512061,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5908,"Index":898,"Attempt":0,"Launch Time":1427397601602,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5904,"Index":894,"Attempt":0,"Launch Time":1427397601595,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601602,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":361575,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5909,"Index":899,"Attempt":0,"Launch Time":1427397601603,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5903,"Index":893,"Attempt":0,"Launch Time":1427397601595,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601603,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":397645,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5910,"Index":900,"Attempt":0,"Launch Time":1427397601604,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5905,"Index":895,"Attempt":0,"Launch Time":1427397601596,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601604,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":357409,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5911,"Index":901,"Attempt":0,"Launch Time":1427397601605,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5906,"Index":896,"Attempt":0,"Launch Time":1427397601597,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601605,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":358856,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5912,"Index":902,"Attempt":0,"Launch Time":1427397601606,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5862,"Index":852,"Attempt":0,"Launch Time":1427397601509,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601606,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":96,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1052228,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5913,"Index":903,"Attempt":0,"Launch Time":1427397601608,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5907,"Index":897,"Attempt":0,"Launch Time":1427397601599,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601614,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":476214,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5914,"Index":904,"Attempt":0,"Launch Time":1427397601618,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5911,"Index":901,"Attempt":0,"Launch Time":1427397601605,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601618,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":303230,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5915,"Index":905,"Attempt":0,"Launch Time":1427397601620,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5913,"Index":903,"Attempt":0,"Launch Time":1427397601608,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601620,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9608,"Shuffle Write Time":288534,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5916,"Index":906,"Attempt":0,"Launch Time":1427397601624,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5912,"Index":902,"Attempt":0,"Launch Time":1427397601606,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601624,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":371898,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5917,"Index":907,"Attempt":0,"Launch Time":1427397601624,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5914,"Index":904,"Attempt":0,"Launch Time":1427397601618,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601625,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":296050,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5918,"Index":908,"Attempt":0,"Launch Time":1427397601626,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5908,"Index":898,"Attempt":0,"Launch Time":1427397601602,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601626,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":24,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":355942,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5919,"Index":909,"Attempt":0,"Launch Time":1427397601627,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5915,"Index":905,"Attempt":0,"Launch Time":1427397601620,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601628,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":342350,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5920,"Index":910,"Attempt":0,"Launch Time":1427397601631,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5917,"Index":907,"Attempt":0,"Launch Time":1427397601624,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601631,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":276717,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5921,"Index":911,"Attempt":0,"Launch Time":1427397601633,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5918,"Index":908,"Attempt":0,"Launch Time":1427397601626,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601633,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":276190,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5922,"Index":912,"Attempt":0,"Launch Time":1427397601634,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5919,"Index":909,"Attempt":0,"Launch Time":1427397601627,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601634,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":317568,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5923,"Index":913,"Attempt":0,"Launch Time":1427397601637,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5920,"Index":910,"Attempt":0,"Launch Time":1427397601631,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601637,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":321600,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5924,"Index":914,"Attempt":0,"Launch Time":1427397601648,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5925,"Index":915,"Attempt":0,"Launch Time":1427397601648,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5926,"Index":916,"Attempt":0,"Launch Time":1427397601648,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5927,"Index":917,"Attempt":0,"Launch Time":1427397601648,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5928,"Index":918,"Attempt":0,"Launch Time":1427397601648,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5929,"Index":919,"Attempt":0,"Launch Time":1427397601648,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5909,"Index":899,"Attempt":0,"Launch Time":1427397601603,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601649,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":44,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":334503,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5921,"Index":911,"Attempt":0,"Launch Time":1427397601633,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601649,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":323503,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5916,"Index":906,"Attempt":0,"Launch Time":1427397601624,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601649,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":423652,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5923,"Index":913,"Attempt":0,"Launch Time":1427397601637,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601649,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":340028,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5865,"Index":855,"Attempt":0,"Launch Time":1427397601514,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601674,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":131,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1710924,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5910,"Index":900,"Attempt":0,"Launch Time":1427397601604,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601674,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":44,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":314658,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5930,"Index":920,"Attempt":0,"Launch Time":1427397601674,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5931,"Index":921,"Attempt":0,"Launch Time":1427397601675,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5932,"Index":922,"Attempt":0,"Launch Time":1427397601676,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5888,"Index":878,"Attempt":0,"Launch Time":1427397601568,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601676,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":7,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":282241,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5928,"Index":918,"Attempt":0,"Launch Time":1427397601648,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601676,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":352028,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5929,"Index":919,"Attempt":0,"Launch Time":1427397601648,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601676,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":325335,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5933,"Index":923,"Attempt":0,"Launch Time":1427397601676,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5926,"Index":916,"Attempt":0,"Launch Time":1427397601648,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601676,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":352533,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5934,"Index":924,"Attempt":0,"Launch Time":1427397601677,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5922,"Index":912,"Attempt":0,"Launch Time":1427397601634,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601677,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":302255,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5935,"Index":925,"Attempt":0,"Launch Time":1427397601677,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5925,"Index":915,"Attempt":0,"Launch Time":1427397601648,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601677,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":434343,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5936,"Index":926,"Attempt":0,"Launch Time":1427397601684,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5934,"Index":924,"Attempt":0,"Launch Time":1427397601677,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601684,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":325924,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5937,"Index":927,"Attempt":0,"Launch Time":1427397601684,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5938,"Index":928,"Attempt":0,"Launch Time":1427397601684,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5935,"Index":925,"Attempt":0,"Launch Time":1427397601677,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601684,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":336611,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5927,"Index":917,"Attempt":0,"Launch Time":1427397601648,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601684,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":27,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":324284,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5939,"Index":929,"Attempt":0,"Launch Time":1427397601685,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5933,"Index":923,"Attempt":0,"Launch Time":1427397601676,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601685,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":339803,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5940,"Index":930,"Attempt":0,"Launch Time":1427397601687,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5932,"Index":922,"Attempt":0,"Launch Time":1427397601676,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601687,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":365739,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5941,"Index":931,"Attempt":0,"Launch Time":1427397601691,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5936,"Index":926,"Attempt":0,"Launch Time":1427397601684,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601691,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":320046,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5942,"Index":932,"Attempt":0,"Launch Time":1427397601692,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5943,"Index":933,"Attempt":0,"Launch Time":1427397601692,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5938,"Index":928,"Attempt":0,"Launch Time":1427397601684,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601692,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":302603,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5939,"Index":929,"Attempt":0,"Launch Time":1427397601685,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601692,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":312120,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5944,"Index":934,"Attempt":0,"Launch Time":1427397601692,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5937,"Index":927,"Attempt":0,"Launch Time":1427397601684,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601692,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":331688,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5945,"Index":935,"Attempt":0,"Launch Time":1427397601697,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5941,"Index":931,"Attempt":0,"Launch Time":1427397601691,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601697,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":317303,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5946,"Index":936,"Attempt":0,"Launch Time":1427397601698,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5943,"Index":933,"Attempt":0,"Launch Time":1427397601692,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601698,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":279709,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5947,"Index":937,"Attempt":0,"Launch Time":1427397601698,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5942,"Index":932,"Attempt":0,"Launch Time":1427397601692,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601699,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":336711,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5948,"Index":938,"Attempt":0,"Launch Time":1427397601699,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5944,"Index":934,"Attempt":0,"Launch Time":1427397601692,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601699,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":340583,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5949,"Index":939,"Attempt":0,"Launch Time":1427397601705,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5945,"Index":935,"Attempt":0,"Launch Time":1427397601697,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601705,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":381968,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5950,"Index":940,"Attempt":0,"Launch Time":1427397601705,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5940,"Index":930,"Attempt":0,"Launch Time":1427397601687,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601705,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":405310,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5951,"Index":941,"Attempt":0,"Launch Time":1427397601706,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5946,"Index":936,"Attempt":0,"Launch Time":1427397601698,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601706,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":401859,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5952,"Index":942,"Attempt":0,"Launch Time":1427397601707,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5948,"Index":938,"Attempt":0,"Launch Time":1427397601699,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601708,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":475668,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5953,"Index":943,"Attempt":0,"Launch Time":1427397601708,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5947,"Index":937,"Attempt":0,"Launch Time":1427397601698,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601708,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":343751,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5954,"Index":944,"Attempt":0,"Launch Time":1427397601713,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5950,"Index":940,"Attempt":0,"Launch Time":1427397601705,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601713,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":317547,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5955,"Index":945,"Attempt":0,"Launch Time":1427397601713,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5949,"Index":939,"Attempt":0,"Launch Time":1427397601705,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601714,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":339417,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5956,"Index":946,"Attempt":0,"Launch Time":1427397601714,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5951,"Index":941,"Attempt":0,"Launch Time":1427397601706,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601714,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":338679,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5957,"Index":947,"Attempt":0,"Launch Time":1427397601714,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5953,"Index":943,"Attempt":0,"Launch Time":1427397601708,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601714,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":314879,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5958,"Index":948,"Attempt":0,"Launch Time":1427397601715,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5952,"Index":942,"Attempt":0,"Launch Time":1427397601707,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601715,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":314719,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5959,"Index":949,"Attempt":0,"Launch Time":1427397601726,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5954,"Index":944,"Attempt":0,"Launch Time":1427397601713,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601726,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":390987,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5960,"Index":950,"Attempt":0,"Launch Time":1427397601727,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5955,"Index":945,"Attempt":0,"Launch Time":1427397601713,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601727,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":318332,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5961,"Index":951,"Attempt":0,"Launch Time":1427397601728,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5956,"Index":946,"Attempt":0,"Launch Time":1427397601714,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601728,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":312216,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5962,"Index":952,"Attempt":0,"Launch Time":1427397601728,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5957,"Index":947,"Attempt":0,"Launch Time":1427397601714,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601728,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":7135543,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5963,"Index":953,"Attempt":0,"Launch Time":1427397601729,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5958,"Index":948,"Attempt":0,"Launch Time":1427397601715,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601729,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":326466,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5964,"Index":954,"Attempt":0,"Launch Time":1427397601732,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5959,"Index":949,"Attempt":0,"Launch Time":1427397601726,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601732,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":275559,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5965,"Index":955,"Attempt":0,"Launch Time":1427397601735,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5961,"Index":951,"Attempt":0,"Launch Time":1427397601728,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601735,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":325253,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5966,"Index":956,"Attempt":0,"Launch Time":1427397601736,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5967,"Index":957,"Attempt":0,"Launch Time":1427397601736,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5960,"Index":950,"Attempt":0,"Launch Time":1427397601727,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601736,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":376175,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5962,"Index":952,"Attempt":0,"Launch Time":1427397601728,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601737,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":344264,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5968,"Index":958,"Attempt":0,"Launch Time":1427397601737,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5963,"Index":953,"Attempt":0,"Launch Time":1427397601729,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601738,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":325364,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5969,"Index":959,"Attempt":0,"Launch Time":1427397601738,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5964,"Index":954,"Attempt":0,"Launch Time":1427397601732,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601738,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":318980,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5970,"Index":960,"Attempt":0,"Launch Time":1427397601742,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5971,"Index":961,"Attempt":0,"Launch Time":1427397601742,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5965,"Index":955,"Attempt":0,"Launch Time":1427397601735,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601742,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":311846,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5966,"Index":956,"Attempt":0,"Launch Time":1427397601736,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601743,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":286302,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5972,"Index":962,"Attempt":0,"Launch Time":1427397601743,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5967,"Index":957,"Attempt":0,"Launch Time":1427397601736,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601743,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":261913,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5973,"Index":963,"Attempt":0,"Launch Time":1427397601744,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5968,"Index":958,"Attempt":0,"Launch Time":1427397601737,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601744,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":322064,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5974,"Index":964,"Attempt":0,"Launch Time":1427397601746,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5969,"Index":959,"Attempt":0,"Launch Time":1427397601738,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601746,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":269804,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5975,"Index":965,"Attempt":0,"Launch Time":1427397601749,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5976,"Index":966,"Attempt":0,"Launch Time":1427397601749,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5971,"Index":961,"Attempt":0,"Launch Time":1427397601742,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601749,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":292976,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5970,"Index":960,"Attempt":0,"Launch Time":1427397601742,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601750,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":300652,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5977,"Index":967,"Attempt":0,"Launch Time":1427397601750,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5972,"Index":962,"Attempt":0,"Launch Time":1427397601743,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601750,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":334258,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5978,"Index":968,"Attempt":0,"Launch Time":1427397601752,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5973,"Index":963,"Attempt":0,"Launch Time":1427397601744,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601752,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":326338,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5979,"Index":969,"Attempt":0,"Launch Time":1427397601754,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5974,"Index":964,"Attempt":0,"Launch Time":1427397601746,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601754,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":717100,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5980,"Index":970,"Attempt":0,"Launch Time":1427397601757,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5975,"Index":965,"Attempt":0,"Launch Time":1427397601749,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601757,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":563338,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5981,"Index":971,"Attempt":0,"Launch Time":1427397601758,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5977,"Index":967,"Attempt":0,"Launch Time":1427397601750,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601758,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":393504,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5982,"Index":972,"Attempt":0,"Launch Time":1427397601759,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5978,"Index":968,"Attempt":0,"Launch Time":1427397601752,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601759,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":336977,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5983,"Index":973,"Attempt":0,"Launch Time":1427397601761,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5979,"Index":969,"Attempt":0,"Launch Time":1427397601754,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601761,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":352564,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5984,"Index":974,"Attempt":0,"Launch Time":1427397601762,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5976,"Index":966,"Attempt":0,"Launch Time":1427397601749,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601762,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":549610,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5985,"Index":975,"Attempt":0,"Launch Time":1427397601765,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5980,"Index":970,"Attempt":0,"Launch Time":1427397601757,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601765,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":315598,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5986,"Index":976,"Attempt":0,"Launch Time":1427397601766,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5982,"Index":972,"Attempt":0,"Launch Time":1427397601759,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601766,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":327931,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5987,"Index":977,"Attempt":0,"Launch Time":1427397601767,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5981,"Index":971,"Attempt":0,"Launch Time":1427397601758,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601767,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":760770,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5988,"Index":978,"Attempt":0,"Launch Time":1427397601769,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5983,"Index":973,"Attempt":0,"Launch Time":1427397601761,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601769,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":376998,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5989,"Index":979,"Attempt":0,"Launch Time":1427397601772,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5985,"Index":975,"Attempt":0,"Launch Time":1427397601765,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601772,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":331460,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5990,"Index":980,"Attempt":0,"Launch Time":1427397601773,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5991,"Index":981,"Attempt":0,"Launch Time":1427397601773,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5986,"Index":976,"Attempt":0,"Launch Time":1427397601766,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601773,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":397959,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5984,"Index":974,"Attempt":0,"Launch Time":1427397601762,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601773,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":335074,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5992,"Index":982,"Attempt":0,"Launch Time":1427397601774,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5987,"Index":977,"Attempt":0,"Launch Time":1427397601767,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601774,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":352278,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5993,"Index":983,"Attempt":0,"Launch Time":1427397601776,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5930,"Index":920,"Attempt":0,"Launch Time":1427397601674,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601776,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":15,"Executor Run Time":76,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":311850,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5994,"Index":984,"Attempt":0,"Launch Time":1427397601780,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5990,"Index":980,"Attempt":0,"Launch Time":1427397601773,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601780,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":308524,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5995,"Index":985,"Attempt":0,"Launch Time":1427397601784,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5931,"Index":921,"Attempt":0,"Launch Time":1427397601675,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601784,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":327681,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5996,"Index":986,"Attempt":0,"Launch Time":1427397601787,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5993,"Index":983,"Attempt":0,"Launch Time":1427397601776,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601787,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":294967,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5997,"Index":987,"Attempt":0,"Launch Time":1427397601789,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5994,"Index":984,"Attempt":0,"Launch Time":1427397601780,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601790,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":392663,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5998,"Index":988,"Attempt":0,"Launch Time":1427397601794,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5924,"Index":914,"Attempt":0,"Launch Time":1427397601648,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601794,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":121,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":335682,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5999,"Index":989,"Attempt":0,"Launch Time":1427397601795,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5996,"Index":986,"Attempt":0,"Launch Time":1427397601787,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601795,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":332819,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":6000,"Index":990,"Attempt":0,"Launch Time":1427397601797,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5997,"Index":987,"Attempt":0,"Launch Time":1427397601789,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601797,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":315687,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":6001,"Index":991,"Attempt":0,"Launch Time":1427397601798,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5995,"Index":985,"Attempt":0,"Launch Time":1427397601784,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601798,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":480787,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":6002,"Index":992,"Attempt":0,"Launch Time":1427397601802,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5998,"Index":988,"Attempt":0,"Launch Time":1427397601794,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601802,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":366030,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":6003,"Index":993,"Attempt":0,"Launch Time":1427397601803,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5999,"Index":989,"Attempt":0,"Launch Time":1427397601795,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601803,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":340388,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":6004,"Index":994,"Attempt":0,"Launch Time":1427397601805,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6000,"Index":990,"Attempt":0,"Launch Time":1427397601797,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601805,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":371736,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":6005,"Index":995,"Attempt":0,"Launch Time":1427397601809,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6001,"Index":991,"Attempt":0,"Launch Time":1427397601798,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601809,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":327916,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":6006,"Index":996,"Attempt":0,"Launch Time":1427397601826,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6002,"Index":992,"Attempt":0,"Launch Time":1427397601802,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601826,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":24,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":330535,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":6007,"Index":997,"Attempt":0,"Launch Time":1427397601828,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6003,"Index":993,"Attempt":0,"Launch Time":1427397601803,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601828,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":25,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":325841,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":6008,"Index":998,"Attempt":0,"Launch Time":1427397601829,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6004,"Index":994,"Attempt":0,"Launch Time":1427397601805,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601829,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":24,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":332396,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":6009,"Index":999,"Attempt":0,"Launch Time":1427397601833,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6006,"Index":996,"Attempt":0,"Launch Time":1427397601826,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601833,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":299895,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6007,"Index":997,"Attempt":0,"Launch Time":1427397601828,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601835,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":299349,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6008,"Index":998,"Attempt":0,"Launch Time":1427397601829,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601835,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":297091,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5988,"Index":978,"Attempt":0,"Launch Time":1427397601769,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601841,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":11,"Executor Run Time":13,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":388214,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6009,"Index":999,"Attempt":0,"Launch Time":1427397601833,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601841,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":384820,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6005,"Index":995,"Attempt":0,"Launch Time":1427397601809,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601841,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":32,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":346979,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5992,"Index":982,"Attempt":0,"Launch Time":1427397601774,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601846,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":37,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":273927,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5989,"Index":979,"Attempt":0,"Launch Time":1427397601772,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601851,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":36,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":339777,"Shuffle Records Written":100}}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5991,"Index":981,"Attempt":0,"Launch Time":1427397601773,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601854,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":42,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":326549,"Shuffle Records Written":100}}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":20,"Stage Attempt ID":0,"Stage Name":"map at :21","Number of Tasks":1000,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:287)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:21)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:28)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:30)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.(:32)\n$line15.$read$$iwC$$iwC$$iwC.(:34)\n$line15.$read$$iwC$$iwC.(:36)\n$line15.$read$$iwC.(:38)\n$line15.$read.(:40)\n$line15.$read$.(:44)\n$line15.$read$.()\n$line15.$eval$.(:7)\n$line15.$eval$.()\n$line15.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Submission Time":1427397599571,"Completion Time":1427397601854,"Accumulables":[]}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":21,"Stage Attempt ID":0,"Stage Name":"next at :26","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":50,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"scala.collection.Iterator$$anon$13.next(Iterator.scala:372)\n$line40.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line40.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:31)\n$line40.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:33)\n$line40.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:35)\n$line40.$read$$iwC$$iwC$$iwC$$iwC.(:37)\n$line40.$read$$iwC$$iwC$$iwC.(:39)\n$line40.$read$$iwC$$iwC.(:41)\n$line40.$read$$iwC.(:43)\n$line40.$read.(:45)\n$line40.$read$.(:49)\n$line40.$read$.()\n$line40.$eval$.(:7)\n$line40.$eval$.()\n$line40.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":21,"Stage Attempt ID":0,"Task Info":{"Task ID":6010,"Index":0,"Attempt":0,"Launch Time":1427397601860,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":21,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6010,"Index":0,"Attempt":0,"Launch Time":1427397601860,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601971,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":109,"Result Size":1060,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":1000,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":191840,"Total Records Read":2000}}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":21,"Stage Attempt ID":0,"Stage Name":"next at :26","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":50,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"scala.collection.Iterator$$anon$13.next(Iterator.scala:372)\n$line40.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line40.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:31)\n$line40.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:33)\n$line40.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:35)\n$line40.$read$$iwC$$iwC$$iwC$$iwC.(:37)\n$line40.$read$$iwC$$iwC$$iwC.(:39)\n$line40.$read$$iwC$$iwC.(:41)\n$line40.$read$$iwC.(:43)\n$line40.$read.(:45)\n$line40.$read$.(:49)\n$line40.$read$.()\n$line40.$eval$.(:7)\n$line40.$eval$.()\n$line40.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Submission Time":1427397601860,"Completion Time":1427397601971,"Accumulables":[]}} -{"Event":"SparkListenerJobEnd","Job ID":10,"Completion Time":1427397601971,"Job Result":{"Result":"JobSucceeded"}} -{"Event":"SparkListenerApplicationEnd","Timestamp":1427397602949} diff --git a/core/src/test/scala/org/apache/spark/JsonTestUtils.scala b/core/src/test/scala/org/apache/spark/JsonTestUtils.scala deleted file mode 100644 index ba367cd476146..0000000000000 --- a/core/src/test/scala/org/apache/spark/JsonTestUtils.scala +++ /dev/null @@ -1,34 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark - -import org.json4s._ -import org.json4s.jackson.JsonMethods - -trait JsonTestUtils { - def assertValidDataInJson(validateJson: JValue, expectedJson: JValue) { - val Diff(c, a, d) = validateJson.diff(expectedJson) - val validatePretty = JsonMethods.pretty(validateJson) - val expectedPretty = JsonMethods.pretty(expectedJson) - val errorMessage = s"Expected:\n$expectedPretty\nFound:\n$validatePretty" - import org.scalactic.TripleEquals._ - assert(c === JNothing, s"$errorMessage\nChanged:\n${JsonMethods.pretty(c)}") - assert(a === JNothing, s"$errorMessage\nAdded:\n${JsonMethods.pretty(a)}") - assert(d === JNothing, s"$errorMessage\nDeleted:\n${JsonMethods.pretty(d)}") - } - -} diff --git a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala index e04a79284175c..b58d62567afe1 100644 --- a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala @@ -28,9 +28,9 @@ import org.scalatest.FunSuite import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, WorkerStateResponse} import org.apache.spark.deploy.master.{ApplicationInfo, DriverInfo, RecoveryState, WorkerInfo} import org.apache.spark.deploy.worker.{DriverRunner, ExecutorRunner} -import org.apache.spark.{JsonTestUtils, SecurityManager, SparkConf} +import org.apache.spark.{SecurityManager, SparkConf} -class JsonProtocolSuite extends FunSuite with JsonTestUtils { +class JsonProtocolSuite extends FunSuite { test("writeApplicationInfo") { val output = JsonProtocol.writeApplicationInfo(createAppInfo()) @@ -136,6 +136,16 @@ class JsonProtocolSuite extends FunSuite with JsonTestUtils { case e: JsonParseException => fail("Invalid Json detected", e) } } + + def assertValidDataInJson(validateJson: JValue, expectedJson: JValue) { + val Diff(c, a, d) = validateJson diff expectedJson + val validatePretty = JsonMethods.pretty(validateJson) + val expectedPretty = JsonMethods.pretty(expectedJson) + val errorMessage = s"Expected:\n$expectedPretty\nFound:\n$validatePretty" + assert(c === JNothing, s"$errorMessage\nChanged:\n${JsonMethods.pretty(c)}") + assert(a === JNothing, s"$errorMessage\nAdded:\n${JsonMethods.pretty(a)}") + assert(d === JNothing, s"$errorMessage\nDelected:\n${JsonMethods.pretty(d)}") + } } object JsonConstants { 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 0744b68c695f9..71ba9c18257b8 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 @@ -14,161 +14,22 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.spark.deploy.history -import java.io.{File, FileInputStream, FileWriter, IOException} -import java.net.{HttpURLConnection, URL} -import javax.servlet.http.{HttpServletRequest, HttpServletResponse} +import javax.servlet.http.HttpServletRequest + +import scala.collection.mutable -import org.apache.commons.io.{FileUtils, IOUtils} -import org.mockito.Mockito.when -import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} +import org.apache.hadoop.fs.Path +import org.mockito.Mockito.{when} +import org.scalatest.FunSuite +import org.scalatest.Matchers import org.scalatest.mock.MockitoSugar -import org.apache.spark.{JsonTestUtils, SecurityManager, SparkConf} import org.apache.spark.ui.SparkUI -/** - * A collection of tests against the historyserver, including comparing responses from the json - * metrics api to a set of known "golden files". If new endpoints / parameters are added, - * cases should be added to this test suite. The expected outcomes can be genered by running - * the HistoryServerSuite.main. Note that this will blindly generate new expectation files matching - * the current behavior -- the developer must verify that behavior is correct. - * - * Similarly, if the behavior is changed, HistoryServerSuite.main can be run to update the - * expectations. However, in general this should be done with extreme caution, as the metrics - * are considered part of Spark's public api. - */ -class HistoryServerSuite extends FunSuite with BeforeAndAfter with Matchers with MockitoSugar - with JsonTestUtils { - - private val logDir = new File("src/test/resources/spark-events") - private val expRoot = new File("src/test/resources/HistoryServerExpectations/") - - private var provider: FsHistoryProvider = null - private var server: HistoryServer = null - private var port: Int = -1 - - def init(): Unit = { - val conf = new SparkConf() - .set("spark.history.fs.logDirectory", logDir.getAbsolutePath) - .set("spark.history.fs.updateInterval", "0") - .set("spark.testing", "true") - provider = new FsHistoryProvider(conf) - provider.checkForLogs() - val securityManager = new SecurityManager(conf) - - server = new HistoryServer(conf, provider, securityManager, 18080) - server.initialize() - server.bind() - port = server.boundPort - } - - def stop(): Unit = { - server.stop() - } - - before { - init() - } - - after{ - stop() - } - - val cases = Seq( - "application list json" -> "applications", - "completed app list json" -> "applications?status=completed", - "running app list json" -> "applications?status=running", - "minDate app list json" -> "applications?minDate=2015-02-10", - "maxDate app list json" -> "applications?maxDate=2015-02-10", - "maxDate2 app list json" -> "applications?maxDate=2015-02-03T10:42:40.000CST", - "one app json" -> "applications/local-1422981780767", - "one app multi-attempt json" -> "applications/local-1426533911241", - "job list json" -> "applications/local-1422981780767/jobs", - "job list from multi-attempt app json(1)" -> "applications/local-1426533911241/1/jobs", - "job list from multi-attempt app json(2)" -> "applications/local-1426533911241/2/jobs", - "one job json" -> "applications/local-1422981780767/jobs/0", - "succeeded job list json" -> "applications/local-1422981780767/jobs?status=succeeded", - "succeeded&failed job list json" -> - "applications/local-1422981780767/jobs?status=succeeded&status=failed", - "executor list json" -> "applications/local-1422981780767/executors", - "stage list json" -> "applications/local-1422981780767/stages", - "complete stage list json" -> "applications/local-1422981780767/stages?status=complete", - "failed stage list json" -> "applications/local-1422981780767/stages?status=failed", - "one stage json" -> "applications/local-1422981780767/stages/1", - "one stage attempt json" -> "applications/local-1422981780767/stages/1/0", - - "stage task summary" -> "applications/local-1427397477963/stages/20/0/taskSummary", - "stage task summary w/ custom quantiles" -> - "applications/local-1427397477963/stages/20/0/taskSummary?quantiles=0.01,0.5,0.99", - - "stage task list" -> "applications/local-1427397477963/stages/20/0/taskList", - "stage task list w/ offset & length" -> - "applications/local-1427397477963/stages/20/0/taskList?offset=10&length=50", - "stage task list w/ sortBy" -> - "applications/local-1427397477963/stages/20/0/taskList?sortBy=DECREASING_RUNTIME", - "stage task list w/ sortBy short names: -runtime" -> - "applications/local-1427397477963/stages/20/0/taskList?sortBy=-runtime", - "stage task list w/ sortBy short names: runtime" -> - "applications/local-1427397477963/stages/20/0/taskList?sortBy=runtime", - - "stage list with accumulable json" -> "applications/local-1426533911241/1/stages", - "stage with accumulable json" -> "applications/local-1426533911241/1/stages/0/0", - "stage task list from multi-attempt app json(1)" -> - "applications/local-1426533911241/1/stages/0/0/taskList", - "stage task list from multi-attempt app json(2)" -> - "applications/local-1426533911241/2/stages/0/0/taskList", - - "rdd list storage json" -> "applications/local-1422981780767/storage/rdd", - "one rdd storage json" -> "applications/local-1422981780767/storage/rdd/0" - ) - - // run a bunch of characterization tests -- just verify the behavior is the same as what is saved - // in the test resource folder - cases.foreach { case (name, path) => - test(name) { - val (code, jsonOpt, errOpt) = getContentAndCode(path) - code should be (HttpServletResponse.SC_OK) - jsonOpt should be ('defined) - errOpt should be (None) - val json = jsonOpt.get - val exp = IOUtils.toString(new FileInputStream( - new File(expRoot, path + "/json_expectation"))) - // compare the ASTs so formatting differences don't cause failures - import org.json4s._ - import org.json4s.jackson.JsonMethods._ - val jsonAst = parse(json) - val expAst = parse(exp) - assertValidDataInJson(jsonAst, expAst) - } - } - - test("response codes on bad paths") { - val badAppId = getContentAndCode("applications/foobar") - badAppId._1 should be (HttpServletResponse.SC_NOT_FOUND) - badAppId._3 should be (Some("unknown app: foobar")) - - val badStageId = getContentAndCode("applications/local-1422981780767/stages/12345") - badStageId._1 should be (HttpServletResponse.SC_NOT_FOUND) - badStageId._3 should be (Some("unknown stage: 12345")) - - val badStageAttemptId = getContentAndCode("applications/local-1422981780767/stages/1/1") - badStageAttemptId._1 should be (HttpServletResponse.SC_NOT_FOUND) - badStageAttemptId._3 should be (Some("unknown attempt for stage 1. Found attempts: [0]")) - - val badStageId2 = getContentAndCode("applications/local-1422981780767/stages/flimflam") - badStageId2._1 should be (HttpServletResponse.SC_NOT_FOUND) - // will take some mucking w/ jersey to get a better error msg in this case - - val badQuantiles = getContentAndCode( - "applications/local-1427397477963/stages/20/0/taskSummary?quantiles=foo,0.1") - badQuantiles._1 should be (HttpServletResponse.SC_BAD_REQUEST) - badQuantiles._3 should be (Some("Bad value for parameter \"quantiles\". Expected a double, " + - "got \"foo\"")) - - getContentAndCode("foobar")._1 should be (HttpServletResponse.SC_NOT_FOUND) - } +class HistoryServerSuite extends FunSuite with Matchers with MockitoSugar { test("generate history page with relative links") { val historyServer = mock[HistoryServer] @@ -193,70 +54,4 @@ class HistoryServerSuite extends FunSuite with BeforeAndAfter with Matchers with } yield (attrs.toString) justHrefs should contain(link) } - - def getContentAndCode(path: String, port: Int = port): (Int, Option[String], Option[String]) = { - HistoryServerSuite.getContentAndCode(new URL(s"http://localhost:$port/json/v1/$path")) - } - - def getUrl(path: String): String = { - HistoryServerSuite.getUrl(new URL(s"http://localhost:$port/json/v1/$path")) - } - - def generateExpectation(path: String): Unit = { - val json = getUrl(path) - val dir = new File(expRoot, path) - dir.mkdirs() - val out = new FileWriter(new File(dir, "json_expectation")) - out.write(json) - out.close() - } -} - -object HistoryServerSuite { - def main(args: Array[String]): Unit = { - // generate the "expected" results for the characterization tests. Just blindly assume the - // current behavior is correct, and write out the returned json to the test/resource files - - val suite = new HistoryServerSuite - FileUtils.deleteDirectory(suite.expRoot) - suite.expRoot.mkdirs() - try { - suite.init() - suite.cases.foreach { case (_, path) => - suite.generateExpectation(path) - } - } finally { - suite.stop() - } - } - - def getContentAndCode(url: URL): (Int, Option[String], Option[String]) = { - val connection = url.openConnection().asInstanceOf[HttpURLConnection] - connection.setRequestMethod("GET") - connection.connect() - val code = connection.getResponseCode() - val inString = try { - val in = Option(connection.getInputStream()) - in.map{IOUtils.toString} - } catch { - case io: IOException => None - } - val errString = try { - val err = Option(connection.getErrorStream()) - err.map{IOUtils.toString} - } catch { - case io: IOException => None - } - (code, inString, errString) - } - - def getUrl(path: URL): String = { - val (code, resultOpt, error) = getContentAndCode(path) - if (code == 200) { - resultOpt.get - } else { - throw new RuntimeException( - "got code: " + code + " when getting " + path + " w/ error: " + error) - } - } } diff --git a/core/src/test/scala/org/apache/spark/status/api/v1/SimpleDateParamTest.scala b/core/src/test/scala/org/apache/spark/status/api/v1/SimpleDateParamTest.scala deleted file mode 100644 index 5274df904d395..0000000000000 --- a/core/src/test/scala/org/apache/spark/status/api/v1/SimpleDateParamTest.scala +++ /dev/null @@ -1,29 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.status.api.v1 - -import org.scalatest.{Matchers, FunSuite} - -class SimpleDateParamTest extends FunSuite with Matchers { - - test("date parsing") { - new SimpleDateParam("2015-02-20T23:21:17.190GMT").timestamp should be (1424474477190L) - new SimpleDateParam("2015-02-20T17:21:17.190CST").timestamp should be (1424474477190L) - new SimpleDateParam("2015-02-20").timestamp should be (1424390400000L) // GMT - } - -} diff --git a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala index 117b2c3960820..d53d7f3ba5ae7 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala @@ -18,13 +18,11 @@ package org.apache.spark.ui import java.net.{HttpURLConnection, URL} -import javax.servlet.http.{HttpServletResponse, HttpServletRequest} +import javax.servlet.http.HttpServletRequest import scala.collection.JavaConversions._ import scala.xml.Node -import org.json4s._ -import org.json4s.jackson.JsonMethods import org.openqa.selenium.htmlunit.HtmlUnitDriver import org.openqa.selenium.{By, WebDriver} import org.scalatest._ @@ -35,9 +33,8 @@ import org.scalatest.time.SpanSugar._ import org.apache.spark.LocalSparkContext._ import org.apache.spark._ import org.apache.spark.api.java.StorageLevels -import org.apache.spark.deploy.history.HistoryServerSuite import org.apache.spark.shuffle.FetchFailedException -import org.apache.spark.status.api.v1.{JacksonMessageWriter, StageStatus} + /** * Selenium tests for the Spark Web UI. @@ -45,8 +42,6 @@ import org.apache.spark.status.api.v1.{JacksonMessageWriter, StageStatus} class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with BeforeAndAfterAll { implicit var webDriver: WebDriver = _ - implicit val formats = DefaultFormats - override def beforeAll(): Unit = { webDriver = new HtmlUnitDriver @@ -81,42 +76,28 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before val rdd = sc.parallelize(Seq(1, 2, 3)) rdd.persist(StorageLevels.DISK_ONLY).count() eventually(timeout(5 seconds), interval(50 milliseconds)) { - goToUi(ui, "/storage") + go to (ui.appUIAddress.stripSuffix("/") + "/storage") val tableRowText = findAll(cssSelector("#storage-by-rdd-table td")).map(_.text).toSeq tableRowText should contain (StorageLevels.DISK_ONLY.description) } eventually(timeout(5 seconds), interval(50 milliseconds)) { - goToUi(ui, "/storage/rdd/?id=0") + go to (ui.appUIAddress.stripSuffix("/") + "/storage/rdd/?id=0") val tableRowText = findAll(cssSelector("#rdd-storage-by-block-table td")).map(_.text).toSeq tableRowText should contain (StorageLevels.DISK_ONLY.description) } - val storageJson = getJson(ui, "storage/rdd") - storageJson.children.length should be (1) - (storageJson \ "storageLevel").extract[String] should be (StorageLevels.DISK_ONLY.description) - val rddJson = getJson(ui, "storage/rdd/0") - (rddJson \ "storageLevel").extract[String] should be (StorageLevels.DISK_ONLY.description) - rdd.unpersist() rdd.persist(StorageLevels.MEMORY_ONLY).count() eventually(timeout(5 seconds), interval(50 milliseconds)) { - goToUi(ui, "/storage") + go to (ui.appUIAddress.stripSuffix("/") + "/storage") val tableRowText = findAll(cssSelector("#storage-by-rdd-table td")).map(_.text).toSeq tableRowText should contain (StorageLevels.MEMORY_ONLY.description) } eventually(timeout(5 seconds), interval(50 milliseconds)) { - goToUi(ui, "/storage/rdd/?id=0") + go to (ui.appUIAddress.stripSuffix("/") + "/storage/rdd/?id=0") val tableRowText = findAll(cssSelector("#rdd-storage-by-block-table td")).map(_.text).toSeq tableRowText should contain (StorageLevels.MEMORY_ONLY.description) } - - val updatedStorageJson = getJson(ui, "storage/rdd") - updatedStorageJson.children.length should be (1) - (updatedStorageJson \ "storageLevel").extract[String] should be ( - StorageLevels.MEMORY_ONLY.description) - val updatedRddJson = getJson(ui, "storage/rdd/0") - (updatedRddJson \ "storageLevel").extract[String] should be ( - StorageLevels.MEMORY_ONLY.description) } } @@ -127,13 +108,10 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before sc.parallelize(1 to 10).map { x => throw new Exception()}.collect() } eventually(timeout(5 seconds), interval(50 milliseconds)) { - goToUi(sc, "/stages") + go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/stages") find(id("active")) should be(None) // Since we hide empty tables find(id("failed")).get.text should be("Failed Stages (1)") } - val stageJson = getJson(sc.ui.get, "stages") - stageJson.children.length should be (1) - (stageJson \ "status").extract[String] should be (StageStatus.FAILED.name()) // Regression test for SPARK-2105 class NotSerializable @@ -142,15 +120,12 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before sc.parallelize(1 to 10).map { x => unserializableObject}.collect() } eventually(timeout(5 seconds), interval(50 milliseconds)) { - goToUi(sc, "/stages") + go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/stages") find(id("active")) should be(None) // Since we hide empty tables // The failure occurs before the stage becomes active, hence we should still show only one // failed stage, not two: find(id("failed")).get.text should be("Failed Stages (1)") } - - val updatedStageJson = getJson(sc.ui.get, "stages") - updatedStageJson should be (stageJson) } } @@ -163,7 +138,7 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before withSpark(newSparkContext(killEnabled = true)) { sc => runSlowJob(sc) eventually(timeout(5 seconds), interval(50 milliseconds)) { - goToUi(sc, "/stages") + go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/stages") assert(hasKillLink) } } @@ -171,7 +146,7 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before withSpark(newSparkContext(killEnabled = false)) { sc => runSlowJob(sc) eventually(timeout(5 seconds), interval(50 milliseconds)) { - goToUi(sc, "/stages") + go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/stages") assert(!hasKillLink) } } @@ -182,7 +157,7 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before // If no job has been run in a job group, then "(Job Group)" should not appear in the header sc.parallelize(Seq(1, 2, 3)).count() eventually(timeout(5 seconds), interval(50 milliseconds)) { - goToUi(sc, "/jobs") + go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/jobs") val tableHeaders = findAll(cssSelector("th")).map(_.text).toSeq tableHeaders should not contain "Job Id (Job Group)" } @@ -190,22 +165,10 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before sc.setJobGroup("my-job-group", "my-job-group-description") sc.parallelize(Seq(1, 2, 3)).count() eventually(timeout(5 seconds), interval(50 milliseconds)) { - goToUi(sc, "/jobs") + go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/jobs") val tableHeaders = findAll(cssSelector("th")).map(_.text).toSeq tableHeaders should contain ("Job Id (Job Group)") } - - val jobJson = getJson(sc.ui.get, "jobs") - for { - job @ JObject(_) <- jobJson - JInt(jobId) <- job \ "jobId" - jobGroup = job \ "jobGroup" - } { - jobId.toInt match { - case 0 => jobGroup should be (JNothing) - case 1 => jobGroup should be (JString("my-job-group")) - } - } } } @@ -232,7 +195,7 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before } mappedData.count() eventually(timeout(5 seconds), interval(50 milliseconds)) { - goToUi(sc, "/jobs") + go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/jobs") find(cssSelector(".stage-progress-cell")).get.text should be ("2/2 (1 failed)") // Ideally, the following test would pass, but currently we overcount completed tasks // if task recomputations occur: @@ -241,32 +204,6 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before // of completed tasks may be higher: find(cssSelector(".progress-cell .progress")).get.text should be ("3/2 (1 failed)") } - val jobJson = getJson(sc.ui.get, "jobs") - (jobJson \ "numTasks").extract[Int]should be (2) - (jobJson \ "numCompletedTasks").extract[Int] should be (3) - (jobJson \ "numFailedTasks").extract[Int] should be (1) - (jobJson \ "numCompletedStages").extract[Int] should be (2) - (jobJson \ "numFailedStages").extract[Int] should be (1) - val stageJson = getJson(sc.ui.get, "stages") - - for { - stage @ JObject(_) <- stageJson - JString(status) <- stage \ "status" - JInt(stageId) <- stage \ "stageId" - JInt(attemptId) <- stage \ "attemptId" - } { - val exp = if (attemptId == 0 && stageId == 1) StageStatus.FAILED else StageStatus.COMPLETE - status should be (exp.name()) - } - - for { - stageId <- 0 to 1 - attemptId <- 0 to 1 - } { - val exp = if (attemptId == 0 && stageId == 1) StageStatus.FAILED else StageStatus.COMPLETE - val stageJson = getJson(sc.ui.get, s"stages/$stageId/$attemptId") - (stageJson \ "status").extract[String] should be (exp.name()) - } } } @@ -281,7 +218,7 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before // Start the job: rdd.countAsync() eventually(timeout(10 seconds), interval(50 milliseconds)) { - goToUi(sc, "/jobs/job/?id=0") + go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/jobs/job/?id=0") find(id("active")).get.text should be ("Active Stages (1)") find(id("pending")).get.text should be ("Pending Stages (2)") // Essentially, we want to check that none of the stage rows show @@ -307,7 +244,7 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before rdd.count() rdd.count() eventually(timeout(10 seconds), interval(50 milliseconds)) { - goToUi(sc, "/jobs") + go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/jobs") // The completed jobs table should have two rows. The first row will be the most recent job: val firstRow = find(cssSelector("tbody tr")).get.underlying val firstRowColumns = firstRow.findElements(By.tagName("td")) @@ -334,7 +271,7 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before rdd.count() rdd.count() eventually(timeout(10 seconds), interval(50 milliseconds)) { - goToUi(sc, "/jobs/job/?id=1") + go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/jobs/job/?id=1") find(id("pending")) should be (None) find(id("active")) should be (None) find(id("failed")) should be (None) @@ -362,7 +299,7 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before rdd.count() rdd.count() eventually(timeout(10 seconds), interval(50 milliseconds)) { - goToUi(sc, "/jobs") + go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/jobs") findAll(cssSelector("tbody tr a")).foreach { link => link.text.toLowerCase should include ("count") link.text.toLowerCase should not include "unknown" @@ -384,7 +321,7 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before } sparkUI.attachTab(newTab) eventually(timeout(10 seconds), interval(50 milliseconds)) { - goToUi(sc, "") + go to (sc.ui.get.appUIAddress.stripSuffix("/")) find(cssSelector("""ul li a[href*="jobs"]""")) should not be(None) find(cssSelector("""ul li a[href*="stages"]""")) should not be(None) find(cssSelector("""ul li a[href*="storage"]""")) should not be(None) @@ -393,12 +330,12 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before } eventually(timeout(10 seconds), interval(50 milliseconds)) { // check whether new page exists - goToUi(sc, "/foo") + go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/foo") find(cssSelector("b")).get.text should include ("html magic") } sparkUI.detachTab(newTab) eventually(timeout(10 seconds), interval(50 milliseconds)) { - goToUi(sc, "") + go to (sc.ui.get.appUIAddress.stripSuffix("/")) find(cssSelector("""ul li a[href*="jobs"]""")) should not be(None) find(cssSelector("""ul li a[href*="stages"]""")) should not be(None) find(cssSelector("""ul li a[href*="storage"]""")) should not be(None) @@ -407,7 +344,7 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before } eventually(timeout(10 seconds), interval(50 milliseconds)) { // check new page not exist - goToUi(sc, "/foo") + go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/foo") find(cssSelector("b")) should be(None) } } @@ -434,163 +371,4 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before } } } - - test("stage & job retention") { - val conf = new SparkConf() - .setMaster("local") - .setAppName("test") - .set("spark.ui.enabled", "true") - .set("spark.ui.port", "0") - .set("spark.ui.retainedStages", "3") - .set("spark.ui.retainedJobs", "2") - val sc = new SparkContext(conf) - assert(sc.ui.isDefined) - - withSpark(sc) { sc => - // run a few jobs & stages ... - (0 until 5).foreach { idx => - // NOTE: if we reverse the order, things don't really behave nicely - // we lose the stage for a job we keep, and then the job doesn't know - // about its last stage - sc.parallelize(idx to (idx + 3)).map(identity).groupBy(identity).map(identity) - .groupBy(identity).count() - sc.parallelize(idx to (idx + 3)).collect() - } - - val expJobInfo = Seq( - ("9", "collect"), - ("8", "count") - ) - - eventually(timeout(1 second), interval(50 milliseconds)) { - goToUi(sc, "/jobs") - // The completed jobs table should have two rows. The first row will be the most recent job: - find("completed-summary").get.text should be ("Completed Jobs: 10, only showing 2") - find("completed").get.text should be ("Completed Jobs (10, only showing 2)") - val rows = findAll(cssSelector("tbody tr")).toIndexedSeq.map{_.underlying} - rows.size should be (expJobInfo.size) - for { - (row, idx) <- rows.zipWithIndex - columns = row.findElements(By.tagName("td")) - id = columns(0).getText() - description = columns(1).getText() - } { - id should be (expJobInfo(idx)._1) - description should include (expJobInfo(idx)._2) - } - } - - val jobsJson = getJson(sc.ui.get, "jobs") - jobsJson.children.size should be (expJobInfo.size) - for { - (job @ JObject(_),idx) <- jobsJson.children.zipWithIndex - id = (job \ "jobId").extract[String] - name = (job \ "name").extract[String] - } { - withClue(s"idx = $idx; id = $id; name = ${name.substring(0,20)}") { - id should be (expJobInfo(idx)._1) - name should include (expJobInfo(idx)._2) - } - } - - // what about when we query for a job that did exist, but has been cleared? - goToUi(sc, "/jobs/job/?id=7") - find("no-info").get.text should be ("No information to display for job 7") - - val badJob = HistoryServerSuite.getContentAndCode(jsonUrl(sc.ui.get, "jobs/7")) - badJob._1 should be (HttpServletResponse.SC_NOT_FOUND) - badJob._2 should be (None) - badJob._3 should be (Some("unknown job: 7")) - - val expStageInfo = Seq( - ("19", "collect"), - ("18", "count"), - ("17", "groupBy") - ) - - eventually(timeout(1 second), interval(50 milliseconds)) { - goToUi(sc, "/stages") - find("completed-summary").get.text should be ("Completed Stages: 20, only showing 3") - find("completed").get.text should be ("Completed Stages (20, only showing 3)") - val rows = findAll(cssSelector("tbody tr")).toIndexedSeq.map{_.underlying} - rows.size should be (3) - for { - (row, idx) <- rows.zipWithIndex - columns = row.findElements(By.tagName("td")) - id = columns(0).getText() - description = columns(1).getText() - } { - id should be (expStageInfo(idx)._1) - description should include (expStageInfo(idx)._2) - } - } - - val stagesJson = getJson(sc.ui.get, "stages") - stagesJson.children.size should be (3) - for { - (stage @ JObject(_), idx) <- stagesJson.children.zipWithIndex - id = (stage \ "stageId").extract[String] - name = (stage \ "name").extract[String] - } { - id should be (expStageInfo(idx)._1) - name should include (expStageInfo(idx)._2) - } - - // nonexistent stage - - goToUi(sc, "/stages/stage/?id=12&attempt=0") - find("no-info").get.text should be ("No information to display for Stage 12 (Attempt 0)") - val badStage = HistoryServerSuite.getContentAndCode(jsonUrl(sc.ui.get,"stages/12/0")) - badStage._1 should be (HttpServletResponse.SC_NOT_FOUND) - badStage._2 should be (None) - badStage._3 should be (Some("unknown stage: 12")) - - val badAttempt = HistoryServerSuite.getContentAndCode(jsonUrl(sc.ui.get,"stages/19/15")) - badAttempt._1 should be (HttpServletResponse.SC_NOT_FOUND) - badAttempt._2 should be (None) - badAttempt._3 should be (Some("unknown attempt for stage 19. Found attempts: [0]")) - - val badStageAttemptList = HistoryServerSuite.getContentAndCode( - jsonUrl(sc.ui.get, "stages/12")) - badStageAttemptList._1 should be (HttpServletResponse.SC_NOT_FOUND) - badStageAttemptList._2 should be (None) - badStageAttemptList._3 should be (Some("unknown stage: 12")) - } - } - - test("live UI json application list") { - withSpark(newSparkContext()) { sc => - val appListRawJson = HistoryServerSuite.getUrl(new URL( - sc.ui.get.appUIAddress + "/json/v1/applications")) - val appListJsonAst = JsonMethods.parse(appListRawJson) - appListJsonAst.children.length should be (1) - val attempts = (appListJsonAst \ "attempts").children - attempts.size should be (1) - (attempts(0) \ "completed").extract[Boolean] should be (false) - parseDate(attempts(0) \ "startTime") should be (sc.startTime) - parseDate(attempts(0) \ "endTime") should be (-1) - val oneAppJsonAst = getJson(sc.ui.get, "") - oneAppJsonAst should be (appListJsonAst.children(0)) - } - } - - def goToUi(sc: SparkContext, path: String): Unit = { - goToUi(sc.ui.get, path) - } - - def goToUi(ui: SparkUI, path: String): Unit = { - go to (ui.appUIAddress.stripSuffix("/") + path) - } - - def parseDate(json: JValue): Long = { - JacksonMessageWriter.makeISODateFormat.parse(json.extract[String]).getTime - } - - def getJson(ui: SparkUI, path: String): JValue = { - JsonMethods.parse(HistoryServerSuite.getUrl(jsonUrl(ui, path))) - } - - def jsonUrl(ui: SparkUI, path: String): URL = { - new URL(ui.appUIAddress + "/json/v1/applications/test/" + path) - } } diff --git a/docs/monitoring.md b/docs/monitoring.md index 1e0fc150862fb..8a85928d6d44d 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -174,80 +174,6 @@ making it easy to identify slow tasks, data skew, etc. Note that the history server only displays completed Spark jobs. One way to signal the completion of a Spark job is to stop the Spark Context explicitly (`sc.stop()`), or in Python using the `with SparkContext() as sc:` to handle the Spark Context setup and tear down, and still show the job history on the UI. -## REST API - -In addition to viewing the metrics in the UI, they are also available as JSON. This gives developers -an easy way to create new visualizations and monitoring tools for Spark. The JSON is available for -both running applications, and in the history server. The endpoints are mounted at `/json/v1`. Eg., -for the history server, they would typically be accessible at `http://:18080/json/v1`, and -for a running application, at `http://localhost:4040/json/v1`. - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
      EndpointMeaning
      /applicationsA list of all applications
      /applications/[app-id]/jobsA list of all jobs for a given application
      /applications/[app-id]/jobs/[job-id]Details for the given job
      /applications/[app-id]/stagesA list of all stages for a given application
      /applications/[app-id]/stages/[stage-id]A list of all attempts for the given stage
      /applications/[app-id]/stages/[stage-id]/[stage-attempt-id]Details for the given stage attempt
      /applications/[app-id]/stages/[stage-id]/[stage-attempt-id]/taskSummarySummary metrics of all tasks in the given stage attempt
      /applications/[app-id]/stages/[stage-id]/[stage-attempt-id]/taskListA list of all tasks for the given stage attempt
      /applications/[app-id]/executorsA list of all executors for the given application
      /applications/[app-id]/storage/rddA list of stored RDDs for the given application
      /applications/[app-id]/storage/rdd/[rdd-id]Details for the storage status of a given RDD
      - -When running on Yarn, each application has multiple attempts, so `[app-id]` is actually -`[app-id]/[attempt-id]` in all cases. - -These endpoints have been strongly versioned to make it easier to develop applications on top. - In particular, Spark guarantees: - -* Endpoints will never be removed from one version -* Individual fields will never be removed for any given endpoint -* New endpoints may be added -* New fields may be added to existing endpoints -* New versions of the api may be added in the future at a separate endpoint (eg., `json/v2`). New versions are *not* required to be backwards compatible. -* Api versions may be dropped, but only after at least one minor release of co-existing with a new api version - -Note that even when examining the UI of a running applications, the `applications/[app-id]` portion is -still required, though there is only one application available. Eg. to see the list of jobs for the -running app, you would go to `http://localhost:4040/json/v1/applications/[app-id]/jobs`. This is to -keep the paths consistent in both modes. - # Metrics Spark has a configurable metrics system based on the diff --git a/pom.xml b/pom.xml index 94cf28332cf47..12ad7c5be4e40 100644 --- a/pom.xml +++ b/pom.xml @@ -623,18 +623,6 @@ - - com.sun.jersey - jersey-server - 1.9 - ${hadoop.deps.scope} - - - com.sun.jersey - jersey-core - 1.9 - ${hadoop.deps.scope} - org.scala-lang scala-compiler From a4669443999dc13a1bb34509c827d8b9096ea84f Mon Sep 17 00:00:00 2001 From: qhuang Date: Tue, 5 May 2015 20:39:56 -0700 Subject: [PATCH 04/75] [SPARK-6841] [SPARKR] add support for mean, median, stdev etc. Moving here from https://github.com/amplab-extras/SparkR-pkg/pull/241 sum() has been implemented. (https://github.com/amplab-extras/SparkR-pkg/pull/242) Now Phase 1: mean, sd, var have been implemented, but some things still need to be improved with the suggestions in https://issues.apache.org/jira/browse/SPARK-6841 Author: qhuang Closes #5446 from hqzizania/R and squashes the following commits: f283572 [qhuang] add test unit for describe() 2e74d5a [qhuang] add describe() DataFrame API --- R/pkg/NAMESPACE | 1 + R/pkg/R/DataFrame.R | 37 ++++++++++++++++++++++++++++++++ R/pkg/R/generics.R | 4 ++++ R/pkg/inst/tests/test_sparkSQL.R | 11 ++++++++++ 4 files changed, 53 insertions(+) diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE index 1fb3311b7f955..528e6608c3c82 100644 --- a/R/pkg/NAMESPACE +++ b/R/pkg/NAMESPACE @@ -13,6 +13,7 @@ exportMethods("cache", "collect", "columns", "count", + "describe", "distinct", "dtypes", "except", diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index 841e77e55e0d8..56c305d912587 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -1276,3 +1276,40 @@ setMethod("saveAsTable", callJMethod(df@sdf, "saveAsTable", tableName, source, jmode, options) }) +#' describe +#' +#' Computes statistics for numeric columns. +#' If no columns are given, this function computes statistics for all numerical columns. +#' +#' @param x A DataFrame to be computed. +#' @param col A string of name +#' @param ... Additional expressions +#' @return A DataFrame +#' @rdname describe +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' describe(df) +#' describe(df, "col1") +#' describe(df, "col1", "col2") +#' } +setMethod("describe", + signature(x = "DataFrame", col = "character"), + function(x, col, ...) { + colList <- list(col, ...) + sdf <- callJMethod(x@sdf, "describe", listToSeq(colList)) + dataFrame(sdf) + }) + +#' @rdname describe +setMethod("describe", + signature(x = "DataFrame"), + function(x) { + colList <- as.list(c(columns(x))) + sdf <- callJMethod(x@sdf, "describe", listToSeq(colList)) + dataFrame(sdf) + }) diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index e88729387ef95..5838955f74dad 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -384,6 +384,10 @@ setGeneric("value", function(bcast) { standardGeneric("value") }) #' @export setGeneric("columns", function(x) {standardGeneric("columns") }) +#' @rdname describe +#' @export +setGeneric("describe", function(x, col, ...) { standardGeneric("describe") }) + #' @rdname schema #' @export setGeneric("dtypes", function(x) { standardGeneric("dtypes") }) diff --git a/R/pkg/inst/tests/test_sparkSQL.R b/R/pkg/inst/tests/test_sparkSQL.R index f82e56fdd8278..7a42e289fcd9e 100644 --- a/R/pkg/inst/tests/test_sparkSQL.R +++ b/R/pkg/inst/tests/test_sparkSQL.R @@ -705,5 +705,16 @@ test_that("parquetFile works with multiple input paths", { expect_true(count(parquetDF) == count(df)*2) }) +test_that("describe() on a DataFrame", { + df <- jsonFile(sqlCtx, jsonPath) + stats <- describe(df, "age") + expect_true(collect(stats)[1, "summary"] == "count") + expect_true(collect(stats)[2, "age"] == 24.5) + expect_true(collect(stats)[3, "age"] == 5.5) + stats <- describe(df) + expect_true(collect(stats)[4, "name"] == "Andy") + expect_true(collect(stats)[5, "age"] == 30.0) +}) + unlink(parquetPath) unlink(jsonPath) From ba2b56614d7ab1bd7409b49e9d85c248d8faa48b Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Tue, 5 May 2015 22:56:01 -0700 Subject: [PATCH 05/75] [SPARK-7358][SQL] Move DataFrame mathfunctions into functions After a discussion on the user mailing list, it was decided to put all UDF's under `o.a.s.sql.functions` cc rxin Author: Burak Yavuz Closes #5923 from brkyvz/move-math-funcs and squashes the following commits: a8dc3f7 [Burak Yavuz] address comments cf7a7bb [Burak Yavuz] [SPARK-7358] Move DataFrame mathfunctions into functions --- python/pyspark/sql/functions.py | 52 ++ python/pyspark/sql/mathfunctions.py | 101 ---- python/pyspark/sql/tests.py | 2 +- .../ExpressionEvaluationSuite.scala | 4 +- .../org/apache/spark/sql/functions.scala | 485 ++++++++++++++++++ .../org/apache/spark/sql/mathfunctions.scala | 383 -------------- .../apache/spark/sql/JavaDataFrameSuite.java | 1 - .../spark/sql/MathExpressionsSuite.scala | 7 +- 8 files changed, 543 insertions(+), 492 deletions(-) delete mode 100644 python/pyspark/sql/mathfunctions.py delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/mathfunctions.scala diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 641220a264295..692af868dd534 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -51,6 +51,19 @@ def _(col): return _ +def _create_binary_mathfunction(name, doc=""): + """ Create a binary mathfunction by name""" + def _(col1, col2): + sc = SparkContext._active_spark_context + # users might write ints for simplicity. This would throw an error on the JVM side. + jc = getattr(sc._jvm.functions, name)(col1._jc if isinstance(col1, Column) else float(col1), + col2._jc if isinstance(col2, Column) else float(col2)) + return Column(jc) + _.__name__ = name + _.__doc__ = doc + return _ + + _functions = { 'lit': 'Creates a :class:`Column` of literal value.', 'col': 'Returns a :class:`Column` based on the given column name.', @@ -63,6 +76,34 @@ def _(col): 'sqrt': 'Computes the square root of the specified float value.', 'abs': 'Computes the absolute value.', + # unary math functions + 'acos': 'Computes the cosine inverse of the given value; the returned angle is in the range' + + '0.0 through pi.', + 'asin': 'Computes the sine inverse of the given value; the returned angle is in the range' + + '-pi/2 through pi/2.', + 'atan': 'Computes the tangent inverse of the given value.', + 'cbrt': 'Computes the cube-root of the given value.', + 'ceil': 'Computes the ceiling of the given value.', + 'cos': 'Computes the cosine of the given value.', + 'cosh': 'Computes the hyperbolic cosine of the given value.', + 'exp': 'Computes the exponential of the given value.', + 'expm1': 'Computes the exponential of the given value minus one.', + 'floor': 'Computes the floor of the given value.', + 'log': 'Computes the natural logarithm of the given value.', + 'log10': 'Computes the logarithm of the given value in Base 10.', + 'log1p': 'Computes the natural logarithm of the given value plus one.', + 'rint': 'Returns the double value that is closest in value to the argument and' + + ' is equal to a mathematical integer.', + 'signum': 'Computes the signum of the given value.', + 'sin': 'Computes the sine of the given value.', + 'sinh': 'Computes the hyperbolic sine of the given value.', + 'tan': 'Computes the tangent of the given value.', + 'tanh': 'Computes the hyperbolic tangent of the given value.', + 'toDegrees': 'Converts an angle measured in radians to an approximately equivalent angle ' + + 'measured in degrees.', + 'toRadians': 'Converts an angle measured in degrees to an approximately equivalent angle ' + + 'measured in radians.', + 'max': 'Aggregate function: returns the maximum value of the expression in a group.', 'min': 'Aggregate function: returns the minimum value of the expression in a group.', 'first': 'Aggregate function: returns the first value in a group.', @@ -74,10 +115,21 @@ def _(col): 'sumDistinct': 'Aggregate function: returns the sum of distinct values in the expression.', } +# math functions that take two arguments as input +_binary_mathfunctions = { + 'atan2': 'Returns the angle theta from the conversion of rectangular coordinates (x, y) to' + + 'polar coordinates (r, theta).', + 'hypot': 'Computes `sqrt(a^2^ + b^2^)` without intermediate overflow or underflow.', + 'pow': 'Returns the value of the first argument raised to the power of the second argument.' +} + for _name, _doc in _functions.items(): globals()[_name] = _create_function(_name, _doc) +for _name, _doc in _binary_mathfunctions.items(): + globals()[_name] = _create_binary_mathfunction(_name, _doc) del _name, _doc __all__ += _functions.keys() +__all__ += _binary_mathfunctions.keys() __all__.sort() diff --git a/python/pyspark/sql/mathfunctions.py b/python/pyspark/sql/mathfunctions.py deleted file mode 100644 index 7dbcab8694293..0000000000000 --- a/python/pyspark/sql/mathfunctions.py +++ /dev/null @@ -1,101 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -""" -A collection of builtin math functions -""" - -from pyspark import SparkContext -from pyspark.sql.dataframe import Column - -__all__ = [] - - -def _create_unary_mathfunction(name, doc=""): - """ Create a unary mathfunction by name""" - def _(col): - sc = SparkContext._active_spark_context - jc = getattr(sc._jvm.mathfunctions, name)(col._jc if isinstance(col, Column) else col) - return Column(jc) - _.__name__ = name - _.__doc__ = doc - return _ - - -def _create_binary_mathfunction(name, doc=""): - """ Create a binary mathfunction by name""" - def _(col1, col2): - sc = SparkContext._active_spark_context - # users might write ints for simplicity. This would throw an error on the JVM side. - if type(col1) is int: - col1 = col1 * 1.0 - if type(col2) is int: - col2 = col2 * 1.0 - jc = getattr(sc._jvm.mathfunctions, name)(col1._jc if isinstance(col1, Column) else col1, - col2._jc if isinstance(col2, Column) else col2) - return Column(jc) - _.__name__ = name - _.__doc__ = doc - return _ - - -# math functions are found under another object therefore, they need to be handled separately -_mathfunctions = { - 'acos': 'Computes the cosine inverse of the given value; the returned angle is in the range' + - '0.0 through pi.', - 'asin': 'Computes the sine inverse of the given value; the returned angle is in the range' + - '-pi/2 through pi/2.', - 'atan': 'Computes the tangent inverse of the given value.', - 'cbrt': 'Computes the cube-root of the given value.', - 'ceil': 'Computes the ceiling of the given value.', - 'cos': 'Computes the cosine of the given value.', - 'cosh': 'Computes the hyperbolic cosine of the given value.', - 'exp': 'Computes the exponential of the given value.', - 'expm1': 'Computes the exponential of the given value minus one.', - 'floor': 'Computes the floor of the given value.', - 'log': 'Computes the natural logarithm of the given value.', - 'log10': 'Computes the logarithm of the given value in Base 10.', - 'log1p': 'Computes the natural logarithm of the given value plus one.', - 'rint': 'Returns the double value that is closest in value to the argument and' + - ' is equal to a mathematical integer.', - 'signum': 'Computes the signum of the given value.', - 'sin': 'Computes the sine of the given value.', - 'sinh': 'Computes the hyperbolic sine of the given value.', - 'tan': 'Computes the tangent of the given value.', - 'tanh': 'Computes the hyperbolic tangent of the given value.', - 'toDeg': 'Converts an angle measured in radians to an approximately equivalent angle ' + - 'measured in degrees.', - 'toRad': 'Converts an angle measured in degrees to an approximately equivalent angle ' + - 'measured in radians.' -} - -# math functions that take two arguments as input -_binary_mathfunctions = { - 'atan2': 'Returns the angle theta from the conversion of rectangular coordinates (x, y) to' + - 'polar coordinates (r, theta).', - 'hypot': 'Computes `sqrt(a^2^ + b^2^)` without intermediate overflow or underflow.', - 'pow': 'Returns the value of the first argument raised to the power of the second argument.' -} - -for _name, _doc in _mathfunctions.items(): - globals()[_name] = _create_unary_mathfunction(_name, _doc) -for _name, _doc in _binary_mathfunctions.items(): - globals()[_name] = _create_binary_mathfunction(_name, _doc) -del _name, _doc -__all__ += _mathfunctions.keys() -__all__ += _binary_mathfunctions.keys() -__all__.sort() diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index 46c4c88e98eb1..b232f3a965526 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -416,7 +416,7 @@ def test_crosstab(self): def test_math_functions(self): df = self.sc.parallelize([Row(a=i, b=2 * i) for i in range(10)]).toDF() - from pyspark.sql import mathfunctions as functions + from pyspark.sql import functions import math def get_values(l): diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala index fa71001c9336e..faaa55aa5e6f1 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala @@ -1217,11 +1217,11 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { unaryMathFunctionEvaluation(Tanh, math.tanh) } - test("toDeg") { + test("toDegrees") { unaryMathFunctionEvaluation(ToDegrees, math.toDegrees) } - test("toRad") { + test("toRadians") { unaryMathFunctionEvaluation(ToRadians, math.toRadians) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala index 7e283393d0563..f2bb4534c75eb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala @@ -24,6 +24,7 @@ import org.apache.spark.annotation.Experimental import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, UnresolvedFunction, Star} import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.mathfuncs._ import org.apache.spark.sql.types._ import org.apache.spark.util.Utils @@ -35,6 +36,7 @@ import org.apache.spark.util.Utils * @groupname agg_funcs Aggregate functions * @groupname sort_funcs Sorting functions * @groupname normal_funcs Non-aggregate functions + * @groupname math_funcs Math functions * @groupname Ungrouped Support functions for DataFrames. */ @Experimental @@ -436,6 +438,489 @@ object functions { */ def upper(e: Column): Column = Upper(e.expr) + ////////////////////////////////////////////////////////////////////////////////////////////// + // Math Functions + ////////////////////////////////////////////////////////////////////////////////////////////// + + /** + * Computes the cosine inverse of the given value; the returned angle is in the range + * 0.0 through pi. + * + * @group math_funcs + */ + def acos(e: Column): Column = Acos(e.expr) + + /** + * Computes the cosine inverse of the given column; the returned angle is in the range + * 0.0 through pi. + * + * @group math_funcs + */ + def acos(columnName: String): Column = acos(Column(columnName)) + + /** + * Computes the sine inverse of the given value; the returned angle is in the range + * -pi/2 through pi/2. + * + * @group math_funcs + */ + def asin(e: Column): Column = Asin(e.expr) + + /** + * Computes the sine inverse of the given column; the returned angle is in the range + * -pi/2 through pi/2. + * + * @group math_funcs + */ + def asin(columnName: String): Column = asin(Column(columnName)) + + /** + * Computes the tangent inverse of the given value. + * + * @group math_funcs + */ + def atan(e: Column): Column = Atan(e.expr) + + /** + * Computes the tangent inverse of the given column. + * + * @group math_funcs + */ + def atan(columnName: String): Column = atan(Column(columnName)) + + /** + * Returns the angle theta from the conversion of rectangular coordinates (x, y) to + * polar coordinates (r, theta). + * + * @group math_funcs + */ + def atan2(l: Column, r: Column): Column = Atan2(l.expr, r.expr) + + /** + * Returns the angle theta from the conversion of rectangular coordinates (x, y) to + * polar coordinates (r, theta). + * + * @group math_funcs + */ + def atan2(l: Column, rightName: String): Column = atan2(l, Column(rightName)) + + /** + * Returns the angle theta from the conversion of rectangular coordinates (x, y) to + * polar coordinates (r, theta). + * + * @group math_funcs + */ + def atan2(leftName: String, r: Column): Column = atan2(Column(leftName), r) + + /** + * Returns the angle theta from the conversion of rectangular coordinates (x, y) to + * polar coordinates (r, theta). + * + * @group math_funcs + */ + def atan2(leftName: String, rightName: String): Column = + atan2(Column(leftName), Column(rightName)) + + /** + * Returns the angle theta from the conversion of rectangular coordinates (x, y) to + * polar coordinates (r, theta). + * + * @group math_funcs + */ + def atan2(l: Column, r: Double): Column = atan2(l, lit(r).expr) + + /** + * Returns the angle theta from the conversion of rectangular coordinates (x, y) to + * polar coordinates (r, theta). + * + * @group math_funcs + */ + def atan2(leftName: String, r: Double): Column = atan2(Column(leftName), r) + + /** + * Returns the angle theta from the conversion of rectangular coordinates (x, y) to + * polar coordinates (r, theta). + * + * @group math_funcs + */ + def atan2(l: Double, r: Column): Column = atan2(lit(l).expr, r) + + /** + * Returns the angle theta from the conversion of rectangular coordinates (x, y) to + * polar coordinates (r, theta). + * + * @group math_funcs + */ + def atan2(l: Double, rightName: String): Column = atan2(l, Column(rightName)) + + /** + * Computes the cube-root of the given value. + * + * @group math_funcs + */ + def cbrt(e: Column): Column = Cbrt(e.expr) + + /** + * Computes the cube-root of the given column. + * + * @group math_funcs + */ + def cbrt(columnName: String): Column = cbrt(Column(columnName)) + + /** + * Computes the ceiling of the given value. + * + * @group math_funcs + */ + def ceil(e: Column): Column = Ceil(e.expr) + + /** + * Computes the ceiling of the given column. + * + * @group math_funcs + */ + def ceil(columnName: String): Column = ceil(Column(columnName)) + + /** + * Computes the cosine of the given value. + * + * @group math_funcs + */ + def cos(e: Column): Column = Cos(e.expr) + + /** + * Computes the cosine of the given column. + * + * @group math_funcs + */ + def cos(columnName: String): Column = cos(Column(columnName)) + + /** + * Computes the hyperbolic cosine of the given value. + * + * @group math_funcs + */ + def cosh(e: Column): Column = Cosh(e.expr) + + /** + * Computes the hyperbolic cosine of the given column. + * + * @group math_funcs + */ + def cosh(columnName: String): Column = cosh(Column(columnName)) + + /** + * Computes the exponential of the given value. + * + * @group math_funcs + */ + def exp(e: Column): Column = Exp(e.expr) + + /** + * Computes the exponential of the given column. + * + * @group math_funcs + */ + def exp(columnName: String): Column = exp(Column(columnName)) + + /** + * Computes the exponential of the given value minus one. + * + * @group math_funcs + */ + def expm1(e: Column): Column = Expm1(e.expr) + + /** + * Computes the exponential of the given column. + * + * @group math_funcs + */ + def expm1(columnName: String): Column = expm1(Column(columnName)) + + /** + * Computes the floor of the given value. + * + * @group math_funcs + */ + def floor(e: Column): Column = Floor(e.expr) + + /** + * Computes the floor of the given column. + * + * @group math_funcs + */ + def floor(columnName: String): Column = floor(Column(columnName)) + + /** + * Computes `sqrt(a^2^ + b^2^)` without intermediate overflow or underflow. + * + * @group math_funcs + */ + def hypot(l: Column, r: Column): Column = Hypot(l.expr, r.expr) + + /** + * Computes `sqrt(a^2^ + b^2^)` without intermediate overflow or underflow. + * + * @group math_funcs + */ + def hypot(l: Column, rightName: String): Column = hypot(l, Column(rightName)) + + /** + * Computes `sqrt(a^2^ + b^2^)` without intermediate overflow or underflow. + * + * @group math_funcs + */ + def hypot(leftName: String, r: Column): Column = hypot(Column(leftName), r) + + /** + * Computes `sqrt(a^2^ + b^2^)` without intermediate overflow or underflow. + * + * @group math_funcs + */ + def hypot(leftName: String, rightName: String): Column = + hypot(Column(leftName), Column(rightName)) + + /** + * Computes `sqrt(a^2^ + b^2^)` without intermediate overflow or underflow. + * + * @group math_funcs + */ + def hypot(l: Column, r: Double): Column = hypot(l, lit(r).expr) + + /** + * Computes `sqrt(a^2^ + b^2^)` without intermediate overflow or underflow. + * + * @group math_funcs + */ + def hypot(leftName: String, r: Double): Column = hypot(Column(leftName), r) + + /** + * Computes `sqrt(a^2^ + b^2^)` without intermediate overflow or underflow. + * + * @group math_funcs + */ + def hypot(l: Double, r: Column): Column = hypot(lit(l).expr, r) + + /** + * Computes `sqrt(a^2^ + b^2^)` without intermediate overflow or underflow. + * + * @group math_funcs + */ + def hypot(l: Double, rightName: String): Column = hypot(l, Column(rightName)) + + /** + * Computes the natural logarithm of the given value. + * + * @group math_funcs + */ + def log(e: Column): Column = Log(e.expr) + + /** + * Computes the natural logarithm of the given column. + * + * @group math_funcs + */ + def log(columnName: String): Column = log(Column(columnName)) + + /** + * Computes the logarithm of the given value in Base 10. + * + * @group math_funcs + */ + def log10(e: Column): Column = Log10(e.expr) + + /** + * Computes the logarithm of the given value in Base 10. + * + * @group math_funcs + */ + def log10(columnName: String): Column = log10(Column(columnName)) + + /** + * Computes the natural logarithm of the given value plus one. + * + * @group math_funcs + */ + def log1p(e: Column): Column = Log1p(e.expr) + + /** + * Computes the natural logarithm of the given column plus one. + * + * @group math_funcs + */ + def log1p(columnName: String): Column = log1p(Column(columnName)) + + /** + * Returns the value of the first argument raised to the power of the second argument. + * + * @group math_funcs + */ + def pow(l: Column, r: Column): Column = Pow(l.expr, r.expr) + + /** + * Returns the value of the first argument raised to the power of the second argument. + * + * @group math_funcs + */ + def pow(l: Column, rightName: String): Column = pow(l, Column(rightName)) + + /** + * Returns the value of the first argument raised to the power of the second argument. + * + * @group math_funcs + */ + def pow(leftName: String, r: Column): Column = pow(Column(leftName), r) + + /** + * Returns the value of the first argument raised to the power of the second argument. + * + * @group math_funcs + */ + def pow(leftName: String, rightName: String): Column = pow(Column(leftName), Column(rightName)) + + /** + * Returns the value of the first argument raised to the power of the second argument. + * + * @group math_funcs + */ + def pow(l: Column, r: Double): Column = pow(l, lit(r).expr) + + /** + * Returns the value of the first argument raised to the power of the second argument. + * + * @group math_funcs + */ + def pow(leftName: String, r: Double): Column = pow(Column(leftName), r) + + /** + * Returns the value of the first argument raised to the power of the second argument. + * + * @group math_funcs + */ + def pow(l: Double, r: Column): Column = pow(lit(l).expr, r) + + /** + * Returns the value of the first argument raised to the power of the second argument. + * + * @group math_funcs + */ + def pow(l: Double, rightName: String): Column = pow(l, Column(rightName)) + + /** + * Returns the double value that is closest in value to the argument and + * is equal to a mathematical integer. + * + * @group math_funcs + */ + def rint(e: Column): Column = Rint(e.expr) + + /** + * Returns the double value that is closest in value to the argument and + * is equal to a mathematical integer. + * + * @group math_funcs + */ + def rint(columnName: String): Column = rint(Column(columnName)) + + /** + * Computes the signum of the given value. + * + * @group math_funcs + */ + def signum(e: Column): Column = Signum(e.expr) + + /** + * Computes the signum of the given column. + * + * @group math_funcs + */ + def signum(columnName: String): Column = signum(Column(columnName)) + + /** + * Computes the sine of the given value. + * + * @group math_funcs + */ + def sin(e: Column): Column = Sin(e.expr) + + /** + * Computes the sine of the given column. + * + * @group math_funcs + */ + def sin(columnName: String): Column = sin(Column(columnName)) + + /** + * Computes the hyperbolic sine of the given value. + * + * @group math_funcs + */ + def sinh(e: Column): Column = Sinh(e.expr) + + /** + * Computes the hyperbolic sine of the given column. + * + * @group math_funcs + */ + def sinh(columnName: String): Column = sinh(Column(columnName)) + + /** + * Computes the tangent of the given value. + * + * @group math_funcs + */ + def tan(e: Column): Column = Tan(e.expr) + + /** + * Computes the tangent of the given column. + * + * @group math_funcs + */ + def tan(columnName: String): Column = tan(Column(columnName)) + + /** + * Computes the hyperbolic tangent of the given value. + * + * @group math_funcs + */ + def tanh(e: Column): Column = Tanh(e.expr) + + /** + * Computes the hyperbolic tangent of the given column. + * + * @group math_funcs + */ + def tanh(columnName: String): Column = tanh(Column(columnName)) + + /** + * Converts an angle measured in radians to an approximately equivalent angle measured in degrees. + * + * @group math_funcs + */ + def toDegrees(e: Column): Column = ToDegrees(e.expr) + + /** + * Converts an angle measured in radians to an approximately equivalent angle measured in degrees. + * + * @group math_funcs + */ + def toDegrees(columnName: String): Column = toDegrees(Column(columnName)) + + /** + * Converts an angle measured in degrees to an approximately equivalent angle measured in radians. + * + * @group math_funcs + */ + def toRadians(e: Column): Column = ToRadians(e.expr) + + /** + * Converts an angle measured in degrees to an approximately equivalent angle measured in radians. + * + * @group math_funcs + */ + def toRadians(columnName: String): Column = toRadians(Column(columnName)) + + ////////////////////////////////////////////////////////////////////////////////////////////// ////////////////////////////////////////////////////////////////////////////////////////////// diff --git a/sql/core/src/main/scala/org/apache/spark/sql/mathfunctions.scala b/sql/core/src/main/scala/org/apache/spark/sql/mathfunctions.scala deleted file mode 100644 index db47480c3864d..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/mathfunctions.scala +++ /dev/null @@ -1,383 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql - -import scala.language.implicitConversions - -import org.apache.spark.annotation.Experimental -import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.expressions.mathfuncs._ -import org.apache.spark.sql.functions.lit - -/** - * :: Experimental :: - * Mathematical Functions available for [[DataFrame]]. - */ -@Experimental -// scalastyle:off -object mathfunctions { -// scalastyle:on - - private[this] implicit def toColumn(expr: Expression): Column = Column(expr) - - /** - * Computes the cosine inverse of the given value; the returned angle is in the range - * 0.0 through pi. - */ - def acos(e: Column): Column = Acos(e.expr) - - /** - * Computes the cosine inverse of the given column; the returned angle is in the range - * 0.0 through pi. - */ - def acos(columnName: String): Column = acos(Column(columnName)) - - /** - * Computes the sine inverse of the given value; the returned angle is in the range - * -pi/2 through pi/2. - */ - def asin(e: Column): Column = Asin(e.expr) - - /** - * Computes the sine inverse of the given column; the returned angle is in the range - * -pi/2 through pi/2. - */ - def asin(columnName: String): Column = asin(Column(columnName)) - - /** - * Computes the tangent inverse of the given value. - */ - def atan(e: Column): Column = Atan(e.expr) - - /** - * Computes the tangent inverse of the given column. - */ - def atan(columnName: String): Column = atan(Column(columnName)) - - /** - * Returns the angle theta from the conversion of rectangular coordinates (x, y) to - * polar coordinates (r, theta). - */ - def atan2(l: Column, r: Column): Column = Atan2(l.expr, r.expr) - - /** - * Returns the angle theta from the conversion of rectangular coordinates (x, y) to - * polar coordinates (r, theta). - */ - def atan2(l: Column, rightName: String): Column = atan2(l, Column(rightName)) - - /** - * Returns the angle theta from the conversion of rectangular coordinates (x, y) to - * polar coordinates (r, theta). - */ - def atan2(leftName: String, r: Column): Column = atan2(Column(leftName), r) - - /** - * Returns the angle theta from the conversion of rectangular coordinates (x, y) to - * polar coordinates (r, theta). - */ - def atan2(leftName: String, rightName: String): Column = - atan2(Column(leftName), Column(rightName)) - - /** - * Returns the angle theta from the conversion of rectangular coordinates (x, y) to - * polar coordinates (r, theta). - */ - def atan2(l: Column, r: Double): Column = atan2(l, lit(r).expr) - - /** - * Returns the angle theta from the conversion of rectangular coordinates (x, y) to - * polar coordinates (r, theta).= - */ - def atan2(leftName: String, r: Double): Column = atan2(Column(leftName), r) - - /** - * Returns the angle theta from the conversion of rectangular coordinates (x, y) to - * polar coordinates (r, theta). - */ - def atan2(l: Double, r: Column): Column = atan2(lit(l).expr, r) - - /** - * Returns the angle theta from the conversion of rectangular coordinates (x, y) to - * polar coordinates (r, theta). - */ - def atan2(l: Double, rightName: String): Column = atan2(l, Column(rightName)) - - /** - * Computes the cube-root of the given value. - */ - def cbrt(e: Column): Column = Cbrt(e.expr) - - /** - * Computes the cube-root of the given column. - */ - def cbrt(columnName: String): Column = cbrt(Column(columnName)) - - /** - * Computes the ceiling of the given value. - */ - def ceil(e: Column): Column = Ceil(e.expr) - - /** - * Computes the ceiling of the given column. - */ - def ceil(columnName: String): Column = ceil(Column(columnName)) - - /** - * Computes the cosine of the given value. - */ - def cos(e: Column): Column = Cos(e.expr) - - /** - * Computes the cosine of the given column. - */ - def cos(columnName: String): Column = cos(Column(columnName)) - - /** - * Computes the hyperbolic cosine of the given value. - */ - def cosh(e: Column): Column = Cosh(e.expr) - - /** - * Computes the hyperbolic cosine of the given column. - */ - def cosh(columnName: String): Column = cosh(Column(columnName)) - - /** - * Computes the exponential of the given value. - */ - def exp(e: Column): Column = Exp(e.expr) - - /** - * Computes the exponential of the given column. - */ - def exp(columnName: String): Column = exp(Column(columnName)) - - /** - * Computes the exponential of the given value minus one. - */ - def expm1(e: Column): Column = Expm1(e.expr) - - /** - * Computes the exponential of the given column. - */ - def expm1(columnName: String): Column = expm1(Column(columnName)) - - /** - * Computes the floor of the given value. - */ - def floor(e: Column): Column = Floor(e.expr) - - /** - * Computes the floor of the given column. - */ - def floor(columnName: String): Column = floor(Column(columnName)) - - /** - * Computes `sqrt(a^2^ + b^2^)` without intermediate overflow or underflow. - */ - def hypot(l: Column, r: Column): Column = Hypot(l.expr, r.expr) - - /** - * Computes `sqrt(a^2^ + b^2^)` without intermediate overflow or underflow. - */ - def hypot(l: Column, rightName: String): Column = hypot(l, Column(rightName)) - - /** - * Computes `sqrt(a^2^ + b^2^)` without intermediate overflow or underflow. - */ - def hypot(leftName: String, r: Column): Column = hypot(Column(leftName), r) - - /** - * Computes `sqrt(a^2^ + b^2^)` without intermediate overflow or underflow. - */ - def hypot(leftName: String, rightName: String): Column = - hypot(Column(leftName), Column(rightName)) - - /** - * Computes `sqrt(a^2^ + b^2^)` without intermediate overflow or underflow. - */ - def hypot(l: Column, r: Double): Column = hypot(l, lit(r).expr) - - /** - * Computes `sqrt(a^2^ + b^2^)` without intermediate overflow or underflow. - */ - def hypot(leftName: String, r: Double): Column = hypot(Column(leftName), r) - - /** - * Computes `sqrt(a^2^ + b^2^)` without intermediate overflow or underflow. - */ - def hypot(l: Double, r: Column): Column = hypot(lit(l).expr, r) - - /** - * Computes `sqrt(a^2^ + b^2^)` without intermediate overflow or underflow. - */ - def hypot(l: Double, rightName: String): Column = hypot(l, Column(rightName)) - - /** - * Computes the natural logarithm of the given value. - */ - def log(e: Column): Column = Log(e.expr) - - /** - * Computes the natural logarithm of the given column. - */ - def log(columnName: String): Column = log(Column(columnName)) - - /** - * Computes the logarithm of the given value in Base 10. - */ - def log10(e: Column): Column = Log10(e.expr) - - /** - * Computes the logarithm of the given value in Base 10. - */ - def log10(columnName: String): Column = log10(Column(columnName)) - - /** - * Computes the natural logarithm of the given value plus one. - */ - def log1p(e: Column): Column = Log1p(e.expr) - - /** - * Computes the natural logarithm of the given column plus one. - */ - def log1p(columnName: String): Column = log1p(Column(columnName)) - - /** - * Returns the value of the first argument raised to the power of the second argument. - */ - def pow(l: Column, r: Column): Column = Pow(l.expr, r.expr) - - /** - * Returns the value of the first argument raised to the power of the second argument. - */ - def pow(l: Column, rightName: String): Column = pow(l, Column(rightName)) - - /** - * Returns the value of the first argument raised to the power of the second argument. - */ - def pow(leftName: String, r: Column): Column = pow(Column(leftName), r) - - /** - * Returns the value of the first argument raised to the power of the second argument. - */ - def pow(leftName: String, rightName: String): Column = pow(Column(leftName), Column(rightName)) - - /** - * Returns the value of the first argument raised to the power of the second argument. - */ - def pow(l: Column, r: Double): Column = pow(l, lit(r).expr) - - /** - * Returns the value of the first argument raised to the power of the second argument. - */ - def pow(leftName: String, r: Double): Column = pow(Column(leftName), r) - - /** - * Returns the value of the first argument raised to the power of the second argument. - */ - def pow(l: Double, r: Column): Column = pow(lit(l).expr, r) - - /** - * Returns the value of the first argument raised to the power of the second argument. - */ - def pow(l: Double, rightName: String): Column = pow(l, Column(rightName)) - - /** - * Returns the double value that is closest in value to the argument and - * is equal to a mathematical integer. - */ - def rint(e: Column): Column = Rint(e.expr) - - /** - * Returns the double value that is closest in value to the argument and - * is equal to a mathematical integer. - */ - def rint(columnName: String): Column = rint(Column(columnName)) - - /** - * Computes the signum of the given value. - */ - def signum(e: Column): Column = Signum(e.expr) - - /** - * Computes the signum of the given column. - */ - def signum(columnName: String): Column = signum(Column(columnName)) - - /** - * Computes the sine of the given value. - */ - def sin(e: Column): Column = Sin(e.expr) - - /** - * Computes the sine of the given column. - */ - def sin(columnName: String): Column = sin(Column(columnName)) - - /** - * Computes the hyperbolic sine of the given value. - */ - def sinh(e: Column): Column = Sinh(e.expr) - - /** - * Computes the hyperbolic sine of the given column. - */ - def sinh(columnName: String): Column = sinh(Column(columnName)) - - /** - * Computes the tangent of the given value. - */ - def tan(e: Column): Column = Tan(e.expr) - - /** - * Computes the tangent of the given column. - */ - def tan(columnName: String): Column = tan(Column(columnName)) - - /** - * Computes the hyperbolic tangent of the given value. - */ - def tanh(e: Column): Column = Tanh(e.expr) - - /** - * Computes the hyperbolic tangent of the given column. - */ - def tanh(columnName: String): Column = tanh(Column(columnName)) - - /** - * Converts an angle measured in radians to an approximately equivalent angle measured in degrees. - */ - def toDeg(e: Column): Column = ToDegrees(e.expr) - - /** - * Converts an angle measured in radians to an approximately equivalent angle measured in degrees. - */ - def toDeg(columnName: String): Column = toDeg(Column(columnName)) - - /** - * Converts an angle measured in degrees to an approximately equivalent angle measured in radians. - */ - def toRad(e: Column): Column = ToRadians(e.expr) - - /** - * Converts an angle measured in degrees to an approximately equivalent angle measured in radians. - */ - def toRad(columnName: String): Column = toRad(Column(columnName)) -} diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java index 58cc8e5be6075..72c42f4fe376b 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java @@ -39,7 +39,6 @@ import java.util.Map; import static org.apache.spark.sql.functions.*; -import static org.apache.spark.sql.mathfunctions.*; public class JavaDataFrameSuite { private transient JavaSparkContext jsc; diff --git a/sql/core/src/test/scala/org/apache/spark/sql/MathExpressionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/MathExpressionsSuite.scala index 9e19bb7482e9b..c4281c4b55c02 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/MathExpressionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/MathExpressionsSuite.scala @@ -19,8 +19,7 @@ package org.apache.spark.sql import java.lang.{Double => JavaDouble} -import org.apache.spark.sql.functions.lit -import org.apache.spark.sql.mathfunctions._ +import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.TestSQLContext import org.apache.spark.sql.test.TestSQLContext.implicits._ @@ -159,11 +158,11 @@ class MathExpressionsSuite extends QueryTest { } test("toDeg") { - testOneToOneMathFunction(toDeg, math.toDegrees) + testOneToOneMathFunction(toDegrees, math.toDegrees) } test("toRad") { - testOneToOneMathFunction(toRad, math.toRadians) + testOneToOneMathFunction(toRadians, math.toRadians) } test("cbrt") { From 7b1457839bdac124a07fd6292f6263f0ded48880 Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Tue, 5 May 2015 22:57:13 -0700 Subject: [PATCH 06/75] [SPARK-6267] [MLLIB] Python API for IsotonicRegression https://issues.apache.org/jira/browse/SPARK-6267 Author: Yanbo Liang Author: Xiangrui Meng Closes #5890 from yanboliang/spark-6267 and squashes the following commits: f20541d [Yanbo Liang] Merge pull request #3 from mengxr/SPARK-6267 7f202f9 [Xiangrui Meng] use Vector to have the best Python 2&3 compatibility 4bccfee [Yanbo Liang] fix doctest ec09412 [Yanbo Liang] fix typos 8214bbb [Yanbo Liang] fix code style 5c8ebe5 [Yanbo Liang] Python API for IsotonicRegression --- .../mllib/api/python/PythonMLLibAPI.scala | 18 +++++ .../mllib/regression/IsotonicRegression.scala | 19 ++++- python/pyspark/mllib/regression.py | 73 ++++++++++++++++++- 3 files changed, 106 insertions(+), 4 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala index b086cec083381..426306d78c1c3 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala @@ -282,6 +282,24 @@ private[python] class PythonMLLibAPI extends Serializable { map(_.asInstanceOf[Object]).asJava } + /** + * Java stub for Python mllib IsotonicRegression.run() + */ + def trainIsotonicRegressionModel( + data: JavaRDD[Vector], + isotonic: Boolean): JList[Object] = { + val isotonicRegressionAlg = new IsotonicRegression().setIsotonic(isotonic) + val input = data.rdd.map { x => + (x(0), x(1), x(2)) + }.persist(StorageLevel.MEMORY_AND_DISK) + try { + val model = isotonicRegressionAlg.run(input) + List[AnyRef](model.boundaryVector, model.predictionVector).asJava + } finally { + data.rdd.unpersist(blocking = false) + } + } + /** * Java stub for Python mllib KMeans.run() */ diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/IsotonicRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/IsotonicRegression.scala index 1d7617046b6c7..be2a00c2dfea4 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/IsotonicRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/IsotonicRegression.scala @@ -21,18 +21,20 @@ import java.io.Serializable import java.lang.{Double => JDouble} import java.util.Arrays.binarySearch +import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer import org.json4s._ import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods._ +import org.apache.spark.SparkContext import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.{JavaDoubleRDD, JavaRDD} +import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.util.{Loader, Saveable} import org.apache.spark.rdd.RDD -import org.apache.spark.SparkContext -import org.apache.spark.sql.{DataFrame, SQLContext} +import org.apache.spark.sql.SQLContext /** * :: Experimental :: @@ -57,6 +59,13 @@ class IsotonicRegressionModel ( assertOrdered(boundaries) assertOrdered(predictions)(predictionOrd) + /** A Java-friendly constructor that takes two Iterable parameters and one Boolean parameter. */ + def this(boundaries: java.lang.Iterable[Double], + predictions: java.lang.Iterable[Double], + isotonic: java.lang.Boolean) = { + this(boundaries.asScala.toArray, predictions.asScala.toArray, isotonic) + } + /** Asserts the input array is monotone with the given ordering. */ private def assertOrdered(xs: Array[Double])(implicit ord: Ordering[Double]): Unit = { var i = 1 @@ -132,6 +141,12 @@ class IsotonicRegressionModel ( } } + /** A convenient method for boundaries called by the Python API. */ + private[mllib] def boundaryVector: Vector = Vectors.dense(boundaries) + + /** A convenient method for boundaries called by the Python API. */ + private[mllib] def predictionVector: Vector = Vectors.dense(predictions) + override def save(sc: SparkContext, path: String): Unit = { IsotonicRegressionModel.SaveLoadV1_0.save(sc, path, boundaries, predictions, isotonic) } diff --git a/python/pyspark/mllib/regression.py b/python/pyspark/mllib/regression.py index 4bc6351bdf02f..41bde2ce3e60b 100644 --- a/python/pyspark/mllib/regression.py +++ b/python/pyspark/mllib/regression.py @@ -18,14 +18,16 @@ import numpy as np from numpy import array +from pyspark import RDD from pyspark.mllib.common import callMLlibFunc, _py2java, _java2py, inherit_doc -from pyspark.mllib.linalg import SparseVector, _convert_to_vector +from pyspark.mllib.linalg import SparseVector, Vectors, _convert_to_vector from pyspark.mllib.util import Saveable, Loader __all__ = ['LabeledPoint', 'LinearModel', 'LinearRegressionModel', 'LinearRegressionWithSGD', 'RidgeRegressionModel', 'RidgeRegressionWithSGD', - 'LassoModel', 'LassoWithSGD'] + 'LassoModel', 'LassoWithSGD', 'IsotonicRegressionModel', + 'IsotonicRegression'] class LabeledPoint(object): @@ -396,6 +398,73 @@ def train(rdd, i): return _regression_train_wrapper(train, RidgeRegressionModel, data, initialWeights) +class IsotonicRegressionModel(Saveable, Loader): + + """Regression model for isotonic regression. + + >>> data = [(1, 0, 1), (2, 1, 1), (3, 2, 1), (1, 3, 1), (6, 4, 1), (17, 5, 1), (16, 6, 1)] + >>> irm = IsotonicRegression.train(sc.parallelize(data)) + >>> irm.predict(3) + 2.0 + >>> irm.predict(5) + 16.5 + >>> irm.predict(sc.parallelize([3, 5])).collect() + [2.0, 16.5] + >>> import os, tempfile + >>> path = tempfile.mkdtemp() + >>> irm.save(sc, path) + >>> sameModel = IsotonicRegressionModel.load(sc, path) + >>> sameModel.predict(3) + 2.0 + >>> sameModel.predict(5) + 16.5 + >>> try: + ... os.removedirs(path) + ... except OSError: + ... pass + """ + + def __init__(self, boundaries, predictions, isotonic): + self.boundaries = boundaries + self.predictions = predictions + self.isotonic = isotonic + + def predict(self, x): + if isinstance(x, RDD): + return x.map(lambda v: self.predict(v)) + return np.interp(x, self.boundaries, self.predictions) + + def save(self, sc, path): + java_boundaries = _py2java(sc, self.boundaries.tolist()) + java_predictions = _py2java(sc, self.predictions.tolist()) + java_model = sc._jvm.org.apache.spark.mllib.regression.IsotonicRegressionModel( + java_boundaries, java_predictions, self.isotonic) + java_model.save(sc._jsc.sc(), path) + + @classmethod + def load(cls, sc, path): + java_model = sc._jvm.org.apache.spark.mllib.regression.IsotonicRegressionModel.load( + sc._jsc.sc(), path) + py_boundaries = _java2py(sc, java_model.boundaryVector()).toArray() + py_predictions = _java2py(sc, java_model.predictionVector()).toArray() + return IsotonicRegressionModel(py_boundaries, py_predictions, java_model.isotonic) + + +class IsotonicRegression(object): + """ + Run IsotonicRegression algorithm to obtain isotonic regression model. + + :param data: RDD of (label, feature, weight) tuples. + :param isotonic: Whether this is isotonic or antitonic. + """ + @classmethod + def train(cls, data, isotonic=True): + """Train a isotonic regression model on the given data.""" + boundaries, predictions = callMLlibFunc("trainIsotonicRegressionModel", + data.map(_convert_to_vector), bool(isotonic)) + return IsotonicRegressionModel(boundaries.toArray(), predictions.toArray(), isotonic) + + def _test(): import doctest from pyspark import SparkContext From 9f019c7223bb79b8d5cd52980b2723a1601d1134 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Tue, 5 May 2015 23:25:28 -0700 Subject: [PATCH 07/75] [SPARK-7384][Core][Tests] Fix flaky tests for distributed mode in BroadcastSuite Fixed the following failure: https://amplab.cs.berkeley.edu/jenkins/job/Spark-1.3-Maven-pre-YARN/hadoop.version=1.0.4,label=centos/452/testReport/junit/org.apache.spark.broadcast/BroadcastSuite/Unpersisting_HttpBroadcast_on_executors_and_driver_in_distributed_mode/ The tests should wait until all slaves are up. Otherwise, there may be only a part of `BlockManager`s registered, and fail the tests. Author: zsxwing Closes #5925 from zsxwing/SPARK-7384 and squashes the following commits: 783cb7b [zsxwing] Add comments for _jobProgressListener and remove postfixOps 1009ef1 [zsxwing] [SPARK-7384][Core][Tests] Fix flaky tests for distributed mode in BroadcastSuite --- .../main/scala/org/apache/spark/SparkContext.scala | 8 +++++--- .../apache/spark/broadcast/BroadcastSuite.scala | 14 +++++++++++++- 2 files changed, 18 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 682dec44ac1a5..b5f040ceb15ca 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -407,15 +407,17 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli if (master == "yarn-client") System.setProperty("SPARK_YARN_MODE", "true") + // "_jobProgressListener" should be set up before creating SparkEnv because when creating + // "SparkEnv", some messages will be posted to "listenerBus" and we should not miss them. + _jobProgressListener = new JobProgressListener(_conf) + listenerBus.addListener(jobProgressListener) + // Create the Spark execution environment (cache, map output tracker, etc) _env = createSparkEnv(_conf, isLocal, listenerBus) SparkEnv.set(_env) _metadataCleaner = new MetadataCleaner(MetadataCleanerType.SPARK_CONTEXT, this.cleanup, _conf) - _jobProgressListener = new JobProgressListener(_conf) - listenerBus.addListener(jobProgressListener) - _statusTracker = new SparkStatusTracker(this) _progressBar = diff --git a/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala b/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala index c8fdfa693912e..06e5f1cf6b96f 100644 --- a/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala +++ b/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala @@ -17,9 +17,11 @@ package org.apache.spark.broadcast +import scala.concurrent.duration._ import scala.util.Random import org.scalatest.{Assertions, FunSuite} +import org.scalatest.concurrent.Eventually._ import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkException, SparkEnv} import org.apache.spark.io.SnappyCompressionCodec @@ -307,7 +309,17 @@ class BroadcastSuite extends FunSuite with LocalSparkContext { removeFromDriver: Boolean) { sc = if (distributed) { - new SparkContext("local-cluster[%d, 1, 512]".format(numSlaves), "test", broadcastConf) + val _sc = + new SparkContext("local-cluster[%d, 1, 512]".format(numSlaves), "test", broadcastConf) + // Wait until all salves are up + eventually(timeout(10.seconds), interval(10.milliseconds)) { + _sc.jobProgressListener.synchronized { + val numBlockManagers = _sc.jobProgressListener.blockManagerIds.size + assert(numBlockManagers == numSlaves + 1, + s"Expect ${numSlaves + 1} block managers, but was ${numBlockManagers}") + } + } + _sc } else { new SparkContext("local", "test", broadcastConf) } From 32cdc815c6fc19b5c8c4eca35f88a61302d67cd5 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 6 May 2015 01:28:43 -0700 Subject: [PATCH 08/75] [SPARK-6940] [MLLIB] Add CrossValidator to Python ML pipeline API Since CrossValidator is a meta algorithm, we copy the implementation in Python. jkbradley Author: Xiangrui Meng Closes #5926 from mengxr/SPARK-6940 and squashes the following commits: 6af181f [Xiangrui Meng] add TODOs 8285134 [Xiangrui Meng] update doc 060f7c3 [Xiangrui Meng] update doctest acac727 [Xiangrui Meng] add keyword args cdddecd [Xiangrui Meng] add CrossValidator in Python --- .../spark/ml/tuning/CrossValidator.scala | 7 +- python/pyspark/ml/pipeline.py | 13 +- python/pyspark/ml/tuning.py | 183 +++++++++++++++++- python/pyspark/ml/wrapper.py | 4 +- 4 files changed, 199 insertions(+), 8 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala b/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala index cee2aa6e85523..9208127eb1d79 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala @@ -52,10 +52,12 @@ private[ml] trait CrossValidatorParams extends Params { def getEstimatorParamMaps: Array[ParamMap] = $(estimatorParamMaps) /** - * param for the evaluator for selection + * param for the evaluator used to select hyper-parameters that maximize the cross-validated + * metric * @group param */ - val evaluator: Param[Evaluator] = new Param(this, "evaluator", "evaluator for selection") + val evaluator: Param[Evaluator] = new Param(this, "evaluator", + "evaluator used to select hyper-parameters that maximize the cross-validated metric") /** @group getParam */ def getEvaluator: Evaluator = $(evaluator) @@ -120,6 +122,7 @@ class CrossValidator extends Estimator[CrossValidatorModel] with CrossValidatorP trainingDataset.unpersist() var i = 0 while (i < numModels) { + // TODO: duplicate evaluator to take extra params from input val metric = eval.evaluate(models(i).transform(validationDataset, epm(i))) logDebug(s"Got metric $metric for model trained with ${epm(i)}.") metrics(i) += metric diff --git a/python/pyspark/ml/pipeline.py b/python/pyspark/ml/pipeline.py index 7b875e4b71254..c1b2077c985cf 100644 --- a/python/pyspark/ml/pipeline.py +++ b/python/pyspark/ml/pipeline.py @@ -22,7 +22,7 @@ from pyspark.mllib.common import inherit_doc -__all__ = ['Estimator', 'Transformer', 'Pipeline', 'PipelineModel', 'Evaluator'] +__all__ = ['Estimator', 'Transformer', 'Pipeline', 'PipelineModel', 'Evaluator', 'Model'] @inherit_doc @@ -70,6 +70,15 @@ def transform(self, dataset, params={}): raise NotImplementedError() +@inherit_doc +class Model(Transformer): + """ + Abstract class for models that are fitted by estimators. + """ + + __metaclass__ = ABCMeta + + @inherit_doc class Pipeline(Estimator): """ @@ -154,7 +163,7 @@ def fit(self, dataset, params={}): @inherit_doc -class PipelineModel(Transformer): +class PipelineModel(Model): """ Represents a compiled pipeline with transformers and fitted models. """ diff --git a/python/pyspark/ml/tuning.py b/python/pyspark/ml/tuning.py index 1773ab5bdcdb1..f6cf2c3439ba5 100644 --- a/python/pyspark/ml/tuning.py +++ b/python/pyspark/ml/tuning.py @@ -16,8 +16,14 @@ # import itertools +import numpy as np -__all__ = ['ParamGridBuilder'] +from pyspark.ml.param import Params, Param +from pyspark.ml import Estimator, Model +from pyspark.ml.util import keyword_only +from pyspark.sql.functions import rand + +__all__ = ['ParamGridBuilder', 'CrossValidator', 'CrossValidatorModel'] class ParamGridBuilder(object): @@ -79,6 +85,179 @@ def build(self): return [dict(zip(keys, prod)) for prod in itertools.product(*grid_values)] +class CrossValidator(Estimator): + """ + K-fold cross validation. + + >>> from pyspark.ml.classification import LogisticRegression + >>> from pyspark.ml.evaluation import BinaryClassificationEvaluator + >>> from pyspark.mllib.linalg import Vectors + >>> dataset = sqlContext.createDataFrame( + ... [(Vectors.dense([0.0, 1.0]), 0.0), + ... (Vectors.dense([1.0, 2.0]), 1.0), + ... (Vectors.dense([0.55, 3.0]), 0.0), + ... (Vectors.dense([0.45, 4.0]), 1.0), + ... (Vectors.dense([0.51, 5.0]), 1.0)] * 10, + ... ["features", "label"]) + >>> lr = LogisticRegression() + >>> grid = ParamGridBuilder().addGrid(lr.maxIter, [0, 1, 5]).build() + >>> evaluator = BinaryClassificationEvaluator() + >>> cv = CrossValidator(estimator=lr, estimatorParamMaps=grid, evaluator=evaluator) + >>> cvModel = cv.fit(dataset) + >>> expected = lr.fit(dataset, {lr.maxIter: 5}).transform(dataset) + >>> cvModel.transform(dataset).collect() == expected.collect() + True + """ + + # a placeholder to make it appear in the generated doc + estimator = Param(Params._dummy(), "estimator", "estimator to be cross-validated") + + # a placeholder to make it appear in the generated doc + estimatorParamMaps = Param(Params._dummy(), "estimatorParamMaps", "estimator param maps") + + # a placeholder to make it appear in the generated doc + evaluator = Param( + Params._dummy(), "evaluator", + "evaluator used to select hyper-parameters that maximize the cross-validated metric") + + # a placeholder to make it appear in the generated doc + numFolds = Param(Params._dummy(), "numFolds", "number of folds for cross validation") + + @keyword_only + def __init__(self, estimator=None, estimatorParamMaps=None, evaluator=None, numFolds=3): + """ + __init__(self, estimator=None, estimatorParamMaps=None, evaluator=None, numFolds=3) + """ + super(CrossValidator, self).__init__() + #: param for estimator to be cross-validated + self.estimator = Param(self, "estimator", "estimator to be cross-validated") + #: param for estimator param maps + self.estimatorParamMaps = Param(self, "estimatorParamMaps", "estimator param maps") + #: param for the evaluator used to select hyper-parameters that + #: maximize the cross-validated metric + self.evaluator = Param( + self, "evaluator", + "evaluator used to select hyper-parameters that maximize the cross-validated metric") + #: param for number of folds for cross validation + self.numFolds = Param(self, "numFolds", "number of folds for cross validation") + self._setDefault(numFolds=3) + kwargs = self.__init__._input_kwargs + self._set(**kwargs) + + @keyword_only + def setParams(self, estimator=None, estimatorParamMaps=None, evaluator=None, numFolds=3): + """ + setParams(self, estimator=None, estimatorParamMaps=None, evaluator=None, numFolds=3): + Sets params for cross validator. + """ + kwargs = self.setParams._input_kwargs + return self._set(**kwargs) + + def setEstimator(self, value): + """ + Sets the value of :py:attr:`estimator`. + """ + self.paramMap[self.estimator] = value + return self + + def getEstimator(self): + """ + Gets the value of estimator or its default value. + """ + return self.getOrDefault(self.estimator) + + def setEstimatorParamMaps(self, value): + """ + Sets the value of :py:attr:`estimatorParamMaps`. + """ + self.paramMap[self.estimatorParamMaps] = value + return self + + def getEstimatorParamMaps(self): + """ + Gets the value of estimatorParamMaps or its default value. + """ + return self.getOrDefault(self.estimatorParamMaps) + + def setEvaluator(self, value): + """ + Sets the value of :py:attr:`evaluator`. + """ + self.paramMap[self.evaluator] = value + return self + + def getEvaluator(self): + """ + Gets the value of evaluator or its default value. + """ + return self.getOrDefault(self.evaluator) + + def setNumFolds(self, value): + """ + Sets the value of :py:attr:`numFolds`. + """ + self.paramMap[self.numFolds] = value + return self + + def getNumFolds(self): + """ + Gets the value of numFolds or its default value. + """ + return self.getOrDefault(self.numFolds) + + def fit(self, dataset, params={}): + paramMap = self.extractParamMap(params) + est = paramMap[self.estimator] + epm = paramMap[self.estimatorParamMaps] + numModels = len(epm) + eva = paramMap[self.evaluator] + nFolds = paramMap[self.numFolds] + h = 1.0 / nFolds + randCol = self.uid + "_rand" + df = dataset.select("*", rand(0).alias(randCol)) + metrics = np.zeros(numModels) + for i in range(nFolds): + validateLB = i * h + validateUB = (i + 1) * h + condition = (df[randCol] >= validateLB) & (df[randCol] < validateUB) + validation = df.filter(condition) + train = df.filter(~condition) + for j in range(numModels): + model = est.fit(train, epm[j]) + # TODO: duplicate evaluator to take extra params from input + metric = eva.evaluate(model.transform(validation, epm[j])) + metrics[j] += metric + bestIndex = np.argmax(metrics) + bestModel = est.fit(dataset, epm[bestIndex]) + return CrossValidatorModel(bestModel) + + +class CrossValidatorModel(Model): + """ + Model from k-fold cross validation. + """ + + def __init__(self, bestModel): + #: best model from cross validation + self.bestModel = bestModel + + def transform(self, dataset, params={}): + return self.bestModel.transform(dataset, params) + + if __name__ == "__main__": import doctest - doctest.testmod() + from pyspark.context import SparkContext + from pyspark.sql import SQLContext + globs = globals().copy() + # The small batch size here ensures that we see multiple batches, + # even in these small test examples: + sc = SparkContext("local[2]", "ml.tuning tests") + sqlContext = SQLContext(sc) + globs['sc'] = sc + globs['sqlContext'] = sqlContext + (failure_count, test_count) = doctest.testmod( + globs=globs, optionflags=doctest.ELLIPSIS) + sc.stop() + if failure_count: + exit(-1) diff --git a/python/pyspark/ml/wrapper.py b/python/pyspark/ml/wrapper.py index 73741c4b40dfb..0634254bbd5cf 100644 --- a/python/pyspark/ml/wrapper.py +++ b/python/pyspark/ml/wrapper.py @@ -20,7 +20,7 @@ from pyspark import SparkContext from pyspark.sql import DataFrame from pyspark.ml.param import Params -from pyspark.ml.pipeline import Estimator, Transformer, Evaluator +from pyspark.ml.pipeline import Estimator, Transformer, Evaluator, Model from pyspark.mllib.common import inherit_doc @@ -133,7 +133,7 @@ def transform(self, dataset, params={}): @inherit_doc -class JavaModel(JavaTransformer): +class JavaModel(Model, JavaTransformer): """ Base class for :py:class:`Model`s that wrap Java/Scala implementations. From 322e7e7f689947aef29909572ee0c0e110ea23b8 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 6 May 2015 08:50:56 -0700 Subject: [PATCH 09/75] [SQL] JavaDoc update for various DataFrame functions. Author: Reynold Xin Closes #5935 from rxin/df-doc1 and squashes the following commits: aaeaadb [Reynold Xin] [SQL] JavaDoc update for various DataFrame functions. --- .../scala/org/apache/spark/sql/Column.scala | 4 ++-- .../org/apache/spark/sql/DataFrame.scala | 16 ++++++++-------- .../spark/sql/DataFrameStatFunctions.scala | 19 +++++++++++++++---- .../org/apache/spark/sql/functions.scala | 14 +++++++------- 4 files changed, 32 insertions(+), 21 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala index c0503bf047052..8eb632d3d600b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala @@ -84,14 +84,14 @@ class Column(protected[sql] val expr: Expression) extends Logging { /** * Inversion of boolean expression, i.e. NOT. - * {{ + * {{{ * // Scala: select rows that are not active (isActive === false) * df.filter( !df("isActive") ) * * // Java: * import static org.apache.spark.sql.functions.*; * df.filter( not(df.col("isActive")) ); - * }} + * }}} * * @group expr_ops */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala index aceb7a9627edd..9d2cd7aae3b82 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala @@ -681,11 +681,11 @@ class DataFrame private[sql]( /** * (Scala-specific) Aggregates on the entire [[DataFrame]] without groups. - * {{ + * {{{ * // df.agg(...) is a shorthand for df.groupBy().agg(...) * df.agg("age" -> "max", "salary" -> "avg") * df.groupBy().agg("age" -> "max", "salary" -> "avg") - * }} + * }}} * @group dfops */ def agg(aggExpr: (String, String), aggExprs: (String, String)*): DataFrame = { @@ -694,33 +694,33 @@ class DataFrame private[sql]( /** * (Scala-specific) Aggregates on the entire [[DataFrame]] without groups. - * {{ + * {{{ * // df.agg(...) is a shorthand for df.groupBy().agg(...) * df.agg(Map("age" -> "max", "salary" -> "avg")) * df.groupBy().agg(Map("age" -> "max", "salary" -> "avg")) - * }} + * }}} * @group dfops */ def agg(exprs: Map[String, String]): DataFrame = groupBy().agg(exprs) /** * (Java-specific) Aggregates on the entire [[DataFrame]] without groups. - * {{ + * {{{ * // df.agg(...) is a shorthand for df.groupBy().agg(...) * df.agg(Map("age" -> "max", "salary" -> "avg")) * df.groupBy().agg(Map("age" -> "max", "salary" -> "avg")) - * }} + * }}} * @group dfops */ def agg(exprs: java.util.Map[String, String]): DataFrame = groupBy().agg(exprs) /** * Aggregates on the entire [[DataFrame]] without groups. - * {{ + * {{{ * // df.agg(...) is a shorthand for df.groupBy().agg(...) * df.agg(max($"age"), avg($"salary")) * df.groupBy().agg(max($"age"), avg($"salary")) - * }} + * }}} * @group dfops */ @scala.annotation.varargs diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala index cb88deab35968..a1e74470afc89 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala @@ -37,7 +37,7 @@ final class DataFrameStatFunctions private[sql](df: DataFrame) { StatFunctions.calculateCov(df, Seq(col1, col2)) } - /* + /** * Calculates the correlation of two columns of a DataFrame. Currently only supports the Pearson * Correlation Coefficient. For Spearman Correlation, consider using RDD methods found in * MLlib's Statistics. @@ -75,7 +75,7 @@ final class DataFrameStatFunctions private[sql](df: DataFrame) { * each row. * @param col2 The name of the second column. Distinct items will make the column names * of the DataFrame. - * @return A Local DataFrame containing the table + * @return A DataFrame containing for the contingency table. */ def crosstab(col1: String, col2: String): DataFrame = { StatFunctions.crossTabulate(df, col1, col2) @@ -110,14 +110,25 @@ final class DataFrameStatFunctions private[sql](df: DataFrame) { } /** - * Python friendly implementation for `freqItems` + * (Scala-specific) Finding frequent items for columns, possibly with false positives. Using the + * frequent element count algorithm described in + * [[http://dx.doi.org/10.1145/762471.762473, proposed by Karp, Schenker, and Papadimitriou]]. + * + * @param cols the names of the columns to search frequent items in. + * @return A Local DataFrame with the Array of frequent items for each column. */ def freqItems(cols: Seq[String], support: Double): DataFrame = { FrequentItems.singlePassFreqItems(df, cols, support) } /** - * Python friendly implementation for `freqItems` with a default `support` of 1%. + * (Scala-specific) Finding frequent items for columns, possibly with false positives. Using the + * frequent element count algorithm described in + * [[http://dx.doi.org/10.1145/762471.762473, proposed by Karp, Schenker, and Papadimitriou]]. + * Uses a `default` support of 1%. + * + * @param cols the names of the columns to search frequent items in. + * @return A Local DataFrame with the Array of frequent items for each column. */ def freqItems(cols: Seq[String]): DataFrame = { FrequentItems.singlePassFreqItems(df, cols, 0.01) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala index f2bb4534c75eb..830b5017717b5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala @@ -22,7 +22,7 @@ import scala.reflect.runtime.universe.{TypeTag, typeTag} import org.apache.spark.annotation.Experimental import org.apache.spark.sql.catalyst.ScalaReflection -import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, UnresolvedFunction, Star} +import org.apache.spark.sql.catalyst.analysis.{UnresolvedFunction, Star} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.mathfuncs._ import org.apache.spark.sql.types._ @@ -86,10 +86,10 @@ object functions { /** * Returns a sort expression based on ascending order of the column. - * {{ + * {{{ * // Sort by dept in ascending order, and then age in descending order. * df.sort(asc("dept"), desc("age")) - * }} + * }}} * * @group sort_funcs */ @@ -97,10 +97,10 @@ object functions { /** * Returns a sort expression based on the descending order of the column. - * {{ + * {{{ * // Sort by dept in ascending order, and then age in descending order. * df.sort(asc("dept"), desc("age")) - * }} + * }}} * * @group sort_funcs */ @@ -353,13 +353,13 @@ object functions { /** * Inversion of boolean expression, i.e. NOT. - * {{ + * {{{ * // Scala: select rows that are not active (isActive === false) * df.filter( !df("isActive") ) * * // Java: * df.filter( not(df.col("isActive")) ); - * }} + * }}} * * @group normal_funcs */ From 150f671c286c57deaf37ab1d8f837d68b5be82a0 Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Wed, 6 May 2015 10:05:10 -0700 Subject: [PATCH 10/75] [SPARK-5456] [SQL] fix decimal compare for jdbc rdd Author: Daoyuan Wang Closes #5803 from adrian-wang/decimalcompare and squashes the following commits: aef0e96 [Daoyuan Wang] add null handle ec455b9 [Daoyuan Wang] fix decimal compare for jdbc rdd --- .../main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala | 8 +++++++- .../test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala | 5 ++++- 2 files changed, 11 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala index 2f6ba48dbc3d9..325a326e2b5b2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala @@ -363,7 +363,13 @@ private[sql] class JDBCRDD( case BooleanConversion => mutableRow.setBoolean(i, rs.getBoolean(pos)) case DateConversion => mutableRow.update(i, DateUtils.fromJavaDate(rs.getDate(pos))) - case DecimalConversion => mutableRow.update(i, rs.getBigDecimal(pos)) + case DecimalConversion => + val decimalVal = rs.getBigDecimal(pos) + if (decimalVal == null) { + mutableRow.update(i, null) + } else { + mutableRow.update(i, Decimal(decimalVal)) + } case DoubleConversion => mutableRow.setDouble(i, rs.getDouble(pos)) case FloatConversion => mutableRow.setFloat(i, rs.getFloat(pos)) case IntegerConversion => mutableRow.setInt(i, rs.getInt(pos)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala index b165ab2b1deb3..3ec17d37c025b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala @@ -271,8 +271,11 @@ class JDBCSuite extends FunSuite with BeforeAndAfter { assert(rows(0).getDouble(0) === 1.00000000000000022) // Yes, I meant ==. assert(rows(0).getDouble(1) === 1.00000011920928955) // Yes, I meant ==. assert(rows(0).getAs[BigDecimal](2) - .equals(new BigDecimal("123456789012345.54321543215432100000"))) + .equals(new BigDecimal("123456789012345.54321543215432100000"))) assert(rows(0).schema.fields(2).dataType === DecimalType(40, 20)) + val compareDecimal = sql("SELECT C FROM flttypes where C > C - 1").collect() + assert(compareDecimal(0).getAs[BigDecimal](0) + .equals(new BigDecimal("123456789012345.54321543215432100000"))) } test("SQL query as table name") { From c3eb441f5487c9b6476e1d6e2a2d852dcc43b986 Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Wed, 6 May 2015 10:30:42 -0700 Subject: [PATCH 11/75] [SPARK-6201] [SQL] promote string and do widen types for IN huangjs Acutally spark sql will first go through analysis period, in which we do widen types and promote strings, and then optimization, where constant IN will be converted into INSET. So it turn out that we only need to fix this for IN. Author: Daoyuan Wang Closes #4945 from adrian-wang/inset and squashes the following commits: 71e05cc [Daoyuan Wang] minor fix 581fa1c [Daoyuan Wang] mysql way f3f7baf [Daoyuan Wang] address comments 5eed4bc [Daoyuan Wang] promote string and do widen types for IN --- .../sql/catalyst/analysis/HiveTypeCoercion.scala | 11 +++++++++++ .../spark/sql/catalyst/optimizer/Optimizer.scala | 4 ++-- .../scala/org/apache/spark/sql/SQLQuerySuite.scala | 9 +++++++++ 3 files changed, 22 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala index 831fb4fe95fe7..96e2aee4de15b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala @@ -69,6 +69,7 @@ trait HiveTypeCoercion { val typeCoercionRules = PropagateTypes :: ConvertNaNs :: + InConversion :: WidenTypes :: PromoteStrings :: DecimalPrecision :: @@ -287,6 +288,16 @@ trait HiveTypeCoercion { } } + /** + * Convert all expressions in in() list to the left operator type + */ + object InConversion extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { + case i @ In(a, b) if b.exists(_.dataType != a.dataType) => + i.makeCopy(Array(a, b.map(Cast(_, a.dataType)))) + } + } + // scalastyle:off /** * Calculates and propagates precision for fixed-precision decimals. Hive has a number of diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 709f7d672d931..e4a60f53d6c09 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -310,8 +310,8 @@ object OptimizeIn extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { case q: LogicalPlan => q transformExpressionsDown { case In(v, list) if !list.exists(!_.isInstanceOf[Literal]) => - val hSet = list.map(e => e.eval(null)) - InSet(v, HashSet() ++ hSet) + val hSet = list.map(e => e.eval(null)) + InSet(v, HashSet() ++ hSet) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 0ab8558c1db13..208cec6a32d4d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -120,6 +120,15 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { Row(1, 1) :: Nil) } + test("SPARK-6201 IN type conversion") { + jsonRDD(sparkContext.parallelize(Seq("{\"a\": \"1\"}}", "{\"a\": \"2\"}}", "{\"a\": \"3\"}}"))) + .registerTempTable("d") + + checkAnswer( + sql("select * from d where d.a in (1,2)"), + Seq(Row("1"), Row("2"))) + } + test("SPARK-3176 Added Parser of SQL ABS()") { checkAnswer( sql("SELECT ABS(-1.3)"), From f2c47082c3412a4cf8cbabe12585147c5ec3ea40 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Wed, 6 May 2015 10:43:00 -0700 Subject: [PATCH 12/75] [SPARK-1442] [SQL] Window Function Support for Spark SQL Adding more information about the implementation... This PR is adding the support of window functions to Spark SQL (specifically OVER and WINDOW clause). For every expression having a OVER clause, we use a WindowExpression as the container of a WindowFunction and the corresponding WindowSpecDefinition (the definition of a window frame, i.e. partition specification, order specification, and frame specification appearing in a OVER clause). # Implementation # The high level work flow of the implementation is described as follows. * Query parsing: In the query parse process, all WindowExpressions are originally placed in the projectList of a Project operator or the aggregateExpressions of an Aggregate operator. It makes our changes to simple and keep all of parsing rules for window functions at a single place (nodesToWindowSpecification). For the WINDOWclause in a query, we use a WithWindowDefinition as the container as the mapping from the name of a window specification to a WindowSpecDefinition. This changes is similar with our common table expression support. * Analysis: The query analysis process has three steps for window functions. * Resolve all WindowSpecReferences by replacing them with WindowSpecReferences according to the mapping table stored in the node of WithWindowDefinition. * Resolve WindowFunctions in the projectList of a Project operator or the aggregateExpressions of an Aggregate operator. For this PR, we use Hive's functions for window functions because we will have a major refactoring of our internal UDAFs and it is better to switch our UDAFs after that refactoring work. * Once we have resolved all WindowFunctions, we will use ResolveWindowFunction to extract WindowExpressions from projectList and aggregateExpressions and then create a Window operator for every distinct WindowSpecDefinition. With this choice, at the execution time, we can rely on the Exchange operator to do all of work on reorganizing the table and we do not need to worry about it in the physical Window operator. An example analyzed plan is shown as follows ``` sql(""" SELECT year, country, product, sales, avg(sales) over(partition by product) avg_product, sum(sales) over(partition by country) sum_country FROM sales ORDER BY year, country, product """).explain(true) == Analyzed Logical Plan == Sort [year#34 ASC,country#35 ASC,product#36 ASC], true Project [year#34,country#35,product#36,sales#37,avg_product#27,sum_country#28] Window [year#34,country#35,product#36,sales#37,avg_product#27], [HiveWindowFunction#org.apache.hadoop.hive.ql.udf.generic.GenericUDAFSum(sales#37) WindowSpecDefinition [country#35], [], ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING AS sum_country#28], WindowSpecDefinition [country#35], [], ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING Window [year#34,country#35,product#36,sales#37], [HiveWindowFunction#org.apache.hadoop.hive.ql.udf.generic.GenericUDAFAverage(sales#37) WindowSpecDefinition [product#36], [], ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING AS avg_product#27], WindowSpecDefinition [product#36], [], ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING Project [year#34,country#35,product#36,sales#37] MetastoreRelation default, sales, None ``` * Query planning: In the process of query planning, we simple generate the physical Window operator based on the logical Window operator. Then, to prepare the executedPlan, the EnsureRequirements rule will add Exchange and Sort operators if necessary. The EnsureRequirements rule will analyze the data properties and try to not add unnecessary shuffle and sort. The physical plan for the above example query is shown below. ``` == Physical Plan == Sort [year#34 ASC,country#35 ASC,product#36 ASC], true Exchange (RangePartitioning [year#34 ASC,country#35 ASC,product#36 ASC], 200), [] Window [year#34,country#35,product#36,sales#37,avg_product#27], [HiveWindowFunction#org.apache.hadoop.hive.ql.udf.generic.GenericUDAFSum(sales#37) WindowSpecDefinition [country#35], [], ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING AS sum_country#28], WindowSpecDefinition [country#35], [], ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING Exchange (HashPartitioning [country#35], 200), [country#35 ASC] Window [year#34,country#35,product#36,sales#37], [HiveWindowFunction#org.apache.hadoop.hive.ql.udf.generic.GenericUDAFAverage(sales#37) WindowSpecDefinition [product#36], [], ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING AS avg_product#27], WindowSpecDefinition [product#36], [], ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING Exchange (HashPartitioning [product#36], 200), [product#36 ASC] HiveTableScan [year#34,country#35,product#36,sales#37], (MetastoreRelation default, sales, None), None ``` * Execution time: At execution time, a physical Window operator buffers all rows in a partition specified in the partition spec of a OVER clause. If necessary, it also maintains a sliding window frame. The current implementation tries to buffer the input parameters of a window function according to the window frame to avoid evaluating a row multiple times. # Future work # Here are three improvements that are not hard to add: * Taking advantage of the window frame specification to reduce the number of rows buffered in the physical Window operator. For some cases, we only need to buffer the rows appearing in the sliding window. But for other cases, we will not be able to reduce the number of rows buffered (e.g. ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING). * When aRAGEN frame is used, for PRECEDING and FOLLOWING, it will be great if the part is an expression (we can start with Literal). So, when the data type of ORDER BY expression is a FractionalType, we can support FractionalType as the type ( still needs to be evaluated as a positive value). * When aRAGEN frame is used, we need to support DateType and TimestampType as the data type of the expression appearing in the order specification. Then, the part of PRECEDING and FOLLOWING can support interval types (once we support them). This is a joint work with guowei2 and yhuai Thanks hbutani hvanhovell for his comments Thanks scwf for his comments and unit tests Author: Yin Huai Closes #5604 from guowei2/windowImplement and squashes the following commits: 76fe1c8 [Yin Huai] Implementation. aa2b0ae [Yin Huai] Tests. --- .../sql/catalyst/analysis/Analyzer.scala | 200 +++ .../sql/catalyst/analysis/CheckAnalysis.scala | 5 + .../sql/catalyst/expressions/Projection.scala | 94 ++ .../expressions/windowExpressions.scala | 340 +++++ .../plans/logical/basicOperators.scala | 30 +- .../spark/sql/catalyst/trees/TreeNode.scala | 23 + .../sql/catalyst/trees/TreeNodeSuite.scala | 44 +- .../spark/sql/execution/SparkStrategies.scala | 2 + .../apache/spark/sql/execution/Window.scala | 480 ++++++ .../execution/HiveCompatibilitySuite.scala | 2 +- .../apache/spark/sql/hive/HiveContext.scala | 1 + .../org/apache/spark/sql/hive/HiveQl.scala | 175 ++- .../org/apache/spark/sql/hive/hiveUdfs.scala | 223 ++- ...ndowing-0-327a8cd39fe30255ff492ee86f660522 | 26 + ...nkNoGBY-0-fef4bf638d52a9a601845347010602fd | 26 + ...rstLast-0-86bb9c97d92fdcd941bcb5143513e2e6 | 26 + ...thWhere-0-84345a9f685ba63b87caa4bb16b122b5 | 5 + ...mWindow-0-6cfc8840d3a4469b0fe11d63182cb59f | 26 + ...tClause-0-88d96a526d3cae6ed8168c5b228974d1 | 26 + ...essions-0-11f6c13cf2710ce7054654cca136e73e | 26 + ...Windows-0-efd1476255eeb1b1961149144f574b7a | 26 + ...untStar-0-1b1fc185c8fddf68e58e92f29052ab2d | 26 + ...stUDAFs-0-6974e5959e41a661e09db18547fef58a | 26 + ...WithGBY-0-67d15ee5915ac64a738fd4b60d75eb35 | 25 + ...tioning-0-cb5618b1e626f3a9d4a030b508b5d251 | 25 + ...stSTATs-0-da0e0cca69e42118a96b8609b8fa5838 | 26 + ...stDISTs-0-672d4cb385b7ced2e446f132474293ad | 26 + ...alViews-0-dea06072f0a64fe4537fae854944ed5a | 78 + ...ndAlias-0-b996a664b06e5741c08079d5c38241bc | 25 + ...Windows-0-227e080e337d734dd88ff814b3b412e4 | 26 + ...FInvoke-0-25912ae7d18c91cc09e17e57968fb5db | 26 + ...nWdwDef-0-88945892370ccbc1125a927a3d55342a | 26 + ...WithSWQ-0-a5a5339330a6a6660d32ccb0cc5d7100 | 25 + ...ecRules-0-fa80b09c99e3c1487de48ea71a88dada | 26 + ...tioning-0-45ccbaf0ee083858f7661c66b11d4768 | 26 + ...usForms-0-3436e50214f9afdec84334e10faa931a | 26 + ...sForms2-0-cba9d84a6b1bb5e36595338d4602377e | 26 + ...derCols-0-7647562850dd367ef1e6c63117805423 | 26 + ...stCount-0-e6e97e884327df86f16b870527ec026c | 26 + ...ForRows-0-99007f45b6406869e048b0e4eb9213f1 | 26 + ...orRange-0-d81a591e90950de291d2f133793e9283 | 26 + ...ForRows-0-fb8648e82e4dd56d6bdcfd739dd1edf0 | 26 + ...orRange-0-3cd04e5f2398853c4850f4f86142bb39 | 26 + ...gregate-0-cb3d2f8c1296044dc2658876bb6103ae | 26 + ...ingUDAF-0-3bde93728761b780a745c2ce0398aa0f | 26 + ...tInSubQ-0-73d5274a21d4f4fd51d2a0f1d98516ce | 26 + ...seAlias-0-4b1ad2515fb079012467e987f484a722 | 26 + ...ngNoGBY-0-70cdc0555a61ef08534a9ebebb95ebbf | 26 + ...iner_sz-0-d3f50875bd5dff172cf813fdb7d738eb | 0 ...iner_sz-1-dda16565b98926fc3587de937b9401c7 | 0 ...ainer_sz-2-374e39786feb745cd70f25be58bfa24 | 0 ...iner_sz-3-d2b5e23edec42a62e61750b110ecbaac | 1 + ...iner_sz-4-50d0c630159068b5b8ccdeb76493f1f7 | 26 + ...iner_sz-5-3f95cd6f4add7a2d0101fe3dd97e5082 | 1 + ...Pruning-0-d3f50875bd5dff172cf813fdb7d738eb | 0 ...Pruning-1-dda16565b98926fc3587de937b9401c7 | 0 ...nPruning-2-374e39786feb745cd70f25be58bfa24 | 0 ...Pruning-3-9294b4a22bc396ff2accabd53c5da98b | 26 + ...nPruning-4-445cab062581c449ceffcb368cdf133 | 26 + ...Pruning-5-89110070c761eafb992eb9315128b53f | 26 + ...stic) 1-0-12a92d8800e0da8b515ba3eaf6a7fd0f | 1049 +++++++++++++ ...stic) 3-0-455e41d9949a2d22bab634fd8e42f2b1 | 1 + ...stic) 4-0-cfad06ae8eba6b047d32a6a61dd59392 | 1 + ...stic) 5-0-d7ca7a61377cef3a9f721a28afdae012 | 1 + ...stic) 6-0-287bcc7679822bc7b684532b267bf11f | 1 + ...nistic)-0-36217f6074daaacddb9fcb50a3f4fb5b | 1049 +++++++++++++ ...nistic)-1-9ee79e711248dd6e0a6ce27e439e55f4 | 1049 +++++++++++++ ...nistic)-2-1e88e0ba414a00195f7ebf6b8600ac04 | 1049 +++++++++++++ ...nistic)-3-34d9ee4120f21d0d0ae914fba0acc60c | 1049 +++++++++++++ ...nistic)-4-dfd39236756a3951bc1ec354799d69e4 | 1049 +++++++++++++ ...nistic)-5-8d0ee3e1605f38214bfad28a5ce897cc | 1 + ...nistic)-0-b7cb25303831392a51cd996e758ac79a | 1049 +++++++++++++ ...nistic)-1-a3d352560ac835993001665db6954965 | 1049 +++++++++++++ ...nistic)-2-fafa16c0f7697ca28aeb6f2698799562 | 1049 +++++++++++++ ...nistic)-3-bda0e7c77d6f4712a03389cb5032bc6d | 1049 +++++++++++++ ...stic) 1-0-2e0cbc2d7c5f16657edacd9e7209e6e7 | 1049 +++++++++++++ ...stic) 1-1-5c5f373e325115d710a7a23fcb1626f1 | 1049 +++++++++++++ ...stic) 1-2-ac487cc1b94130bf9ce00e07c7075f65 | 1049 +++++++++++++ ...stic) 1-3-b82dfa24123047be4b4e3c27c3997d34 | 1049 +++++++++++++ ...istic) 2-0-81bb7f49a55385878637c8aac4d08e5 | 1294 +++++++++++++++++ ...stic) 3-0-58a982694ba2b1e34de82b1de54936a0 | 0 ...stic) 4-0-12cc78f3953c3e6b5411ddc729541bf0 | 474 ++++++ ...nistic)-0-6642a21d87e0401ba1a668ea8b244f0c | 1049 +++++++++++++ ...nistic)-1-2bf20f39e6ffef258858f7943a974e7e | 1049 +++++++++++++ ...nistic)-2-16239d2b069789ba99fbac50c4f0724f | 1049 +++++++++++++ ...nistic)-3-d90b27fca067b0b3c48d873b3ef32af7 | 1049 +++++++++++++ ...nistic)-4-f2e4d659b65a833e9281b6786d3d55c1 | 1049 +++++++++++++ ...g_udaf2-0-96659fde37d7a38ea15b367b47f59ce2 | 0 ...g_udaf2-1-b4bdee4908b1cb8e240c549ae5cfe4c0 | 1 + ...nistic)-0-f498cccf82480be03022d2a36f87651e | 1049 +++++++++++++ ...inistic)-1-6378faf36ffd3f61e61cee6c0cb70e6 | 1049 +++++++++++++ ...nistic)-2-5f0eab306ea3c22b11ace9b542a7ee56 | 1049 +++++++++++++ ...nistic)-3-6f104992e0050576085064815de43194 | 1049 +++++++++++++ ...nistic)-4-cd2e3d2344810cb3ba843d4c01c81d7e | 1049 +++++++++++++ ...nistic)-5-ee44c5cdc80e1c832b702f9fb76d8145 | 1049 +++++++++++++ ...nistic)-6-4d78f7b1d172d20c91f5867bc13a42a0 | 1049 +++++++++++++ ...nistic)-7-20fdc99aa046b2c41d9b85ab338c749c | 1049 +++++++++++++ ...nistic)-8-45a1d7c2aba45d761e19ff4dfdf5463e | 1049 +++++++++++++ .../hive/execution/HiveComparisonTest.scala | 2 +- .../HiveWindowFunctionQuerySuite.scala | 845 +++++++++++ .../sql/hive/execution/SQLQuerySuite.scala | 147 ++ 101 files changed, 34768 insertions(+), 34 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/Window.scala create mode 100644 sql/hive/src/test/resources/golden/windowing.q -- 1. testWindowing-0-327a8cd39fe30255ff492ee86f660522 create mode 100644 sql/hive/src/test/resources/golden/windowing.q -- 10. testHavingWithWindowingCondRankNoGBY-0-fef4bf638d52a9a601845347010602fd create mode 100644 sql/hive/src/test/resources/golden/windowing.q -- 11. testFirstLast-0-86bb9c97d92fdcd941bcb5143513e2e6 create mode 100644 sql/hive/src/test/resources/golden/windowing.q -- 12. testFirstLastWithWhere-0-84345a9f685ba63b87caa4bb16b122b5 create mode 100644 sql/hive/src/test/resources/golden/windowing.q -- 13. testSumWindow-0-6cfc8840d3a4469b0fe11d63182cb59f create mode 100644 sql/hive/src/test/resources/golden/windowing.q -- 14. testNoSortClause-0-88d96a526d3cae6ed8168c5b228974d1 create mode 100644 sql/hive/src/test/resources/golden/windowing.q -- 15. testExpressions-0-11f6c13cf2710ce7054654cca136e73e create mode 100644 sql/hive/src/test/resources/golden/windowing.q -- 16. testMultipleWindows-0-efd1476255eeb1b1961149144f574b7a create mode 100644 sql/hive/src/test/resources/golden/windowing.q -- 17. testCountStar-0-1b1fc185c8fddf68e58e92f29052ab2d create mode 100644 sql/hive/src/test/resources/golden/windowing.q -- 18. testUDAFs-0-6974e5959e41a661e09db18547fef58a create mode 100644 sql/hive/src/test/resources/golden/windowing.q -- 19. testUDAFsWithGBY-0-67d15ee5915ac64a738fd4b60d75eb35 create mode 100644 sql/hive/src/test/resources/golden/windowing.q -- 2. testGroupByWithPartitioning-0-cb5618b1e626f3a9d4a030b508b5d251 create mode 100644 sql/hive/src/test/resources/golden/windowing.q -- 20. testSTATs-0-da0e0cca69e42118a96b8609b8fa5838 create mode 100644 sql/hive/src/test/resources/golden/windowing.q -- 21. testDISTs-0-672d4cb385b7ced2e446f132474293ad create mode 100644 sql/hive/src/test/resources/golden/windowing.q -- 24. testLateralViews-0-dea06072f0a64fe4537fae854944ed5a create mode 100644 sql/hive/src/test/resources/golden/windowing.q -- 26. testGroupByHavingWithSWQAndAlias-0-b996a664b06e5741c08079d5c38241bc create mode 100644 sql/hive/src/test/resources/golden/windowing.q -- 27. testMultipleRangeWindows-0-227e080e337d734dd88ff814b3b412e4 create mode 100644 sql/hive/src/test/resources/golden/windowing.q -- 28. testPartOrderInUDAFInvoke-0-25912ae7d18c91cc09e17e57968fb5db create mode 100644 sql/hive/src/test/resources/golden/windowing.q -- 29. testPartOrderInWdwDef-0-88945892370ccbc1125a927a3d55342a create mode 100644 sql/hive/src/test/resources/golden/windowing.q -- 3. testGroupByHavingWithSWQ-0-a5a5339330a6a6660d32ccb0cc5d7100 create mode 100644 sql/hive/src/test/resources/golden/windowing.q -- 30. testDefaultPartitioningSpecRules-0-fa80b09c99e3c1487de48ea71a88dada create mode 100644 sql/hive/src/test/resources/golden/windowing.q -- 36. testRankWithPartitioning-0-45ccbaf0ee083858f7661c66b11d4768 create mode 100644 sql/hive/src/test/resources/golden/windowing.q -- 37. testPartitioningVariousForms-0-3436e50214f9afdec84334e10faa931a create mode 100644 sql/hive/src/test/resources/golden/windowing.q -- 38. testPartitioningVariousForms2-0-cba9d84a6b1bb5e36595338d4602377e create mode 100644 sql/hive/src/test/resources/golden/windowing.q -- 39. testUDFOnOrderCols-0-7647562850dd367ef1e6c63117805423 create mode 100644 sql/hive/src/test/resources/golden/windowing.q -- 4. testCount-0-e6e97e884327df86f16b870527ec026c create mode 100644 sql/hive/src/test/resources/golden/windowing.q -- 40. testNoBetweenForRows-0-99007f45b6406869e048b0e4eb9213f1 create mode 100644 sql/hive/src/test/resources/golden/windowing.q -- 41. testNoBetweenForRange-0-d81a591e90950de291d2f133793e9283 create mode 100644 sql/hive/src/test/resources/golden/windowing.q -- 42. testUnboundedFollowingForRows-0-fb8648e82e4dd56d6bdcfd739dd1edf0 create mode 100644 sql/hive/src/test/resources/golden/windowing.q -- 43. testUnboundedFollowingForRange-0-3cd04e5f2398853c4850f4f86142bb39 create mode 100644 sql/hive/src/test/resources/golden/windowing.q -- 44. testOverNoPartitionSingleAggregate-0-cb3d2f8c1296044dc2658876bb6103ae create mode 100644 sql/hive/src/test/resources/golden/windowing.q -- 5. testCountWithWindowingUDAF-0-3bde93728761b780a745c2ce0398aa0f create mode 100644 sql/hive/src/test/resources/golden/windowing.q -- 6. testCountInSubQ-0-73d5274a21d4f4fd51d2a0f1d98516ce create mode 100644 sql/hive/src/test/resources/golden/windowing.q -- 8. testMixedCaseAlias-0-4b1ad2515fb079012467e987f484a722 create mode 100644 sql/hive/src/test/resources/golden/windowing.q -- 9. testHavingWithWindowingNoGBY-0-70cdc0555a61ef08534a9ebebb95ebbf create mode 100644 sql/hive/src/test/resources/golden/windowing_adjust_rowcontainer_sz-0-d3f50875bd5dff172cf813fdb7d738eb create mode 100644 sql/hive/src/test/resources/golden/windowing_adjust_rowcontainer_sz-1-dda16565b98926fc3587de937b9401c7 create mode 100644 sql/hive/src/test/resources/golden/windowing_adjust_rowcontainer_sz-2-374e39786feb745cd70f25be58bfa24 create mode 100644 sql/hive/src/test/resources/golden/windowing_adjust_rowcontainer_sz-3-d2b5e23edec42a62e61750b110ecbaac create mode 100644 sql/hive/src/test/resources/golden/windowing_adjust_rowcontainer_sz-4-50d0c630159068b5b8ccdeb76493f1f7 create mode 100644 sql/hive/src/test/resources/golden/windowing_adjust_rowcontainer_sz-5-3f95cd6f4add7a2d0101fe3dd97e5082 create mode 100644 sql/hive/src/test/resources/golden/windowing_columnPruning-0-d3f50875bd5dff172cf813fdb7d738eb create mode 100644 sql/hive/src/test/resources/golden/windowing_columnPruning-1-dda16565b98926fc3587de937b9401c7 create mode 100644 sql/hive/src/test/resources/golden/windowing_columnPruning-2-374e39786feb745cd70f25be58bfa24 create mode 100644 sql/hive/src/test/resources/golden/windowing_columnPruning-3-9294b4a22bc396ff2accabd53c5da98b create mode 100644 sql/hive/src/test/resources/golden/windowing_columnPruning-4-445cab062581c449ceffcb368cdf133 create mode 100644 sql/hive/src/test/resources/golden/windowing_columnPruning-5-89110070c761eafb992eb9315128b53f create mode 100644 sql/hive/src/test/resources/golden/windowing_multipartitioning.q (deterministic) 1-0-12a92d8800e0da8b515ba3eaf6a7fd0f create mode 100644 sql/hive/src/test/resources/golden/windowing_multipartitioning.q (deterministic) 3-0-455e41d9949a2d22bab634fd8e42f2b1 create mode 100644 sql/hive/src/test/resources/golden/windowing_multipartitioning.q (deterministic) 4-0-cfad06ae8eba6b047d32a6a61dd59392 create mode 100644 sql/hive/src/test/resources/golden/windowing_multipartitioning.q (deterministic) 5-0-d7ca7a61377cef3a9f721a28afdae012 create mode 100644 sql/hive/src/test/resources/golden/windowing_multipartitioning.q (deterministic) 6-0-287bcc7679822bc7b684532b267bf11f create mode 100644 sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-0-36217f6074daaacddb9fcb50a3f4fb5b create mode 100644 sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-1-9ee79e711248dd6e0a6ce27e439e55f4 create mode 100644 sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-2-1e88e0ba414a00195f7ebf6b8600ac04 create mode 100644 sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-3-34d9ee4120f21d0d0ae914fba0acc60c create mode 100644 sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-4-dfd39236756a3951bc1ec354799d69e4 create mode 100644 sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-5-8d0ee3e1605f38214bfad28a5ce897cc create mode 100644 sql/hive/src/test/resources/golden/windowing_ntile.q (deterministic)-0-b7cb25303831392a51cd996e758ac79a create mode 100644 sql/hive/src/test/resources/golden/windowing_ntile.q (deterministic)-1-a3d352560ac835993001665db6954965 create mode 100644 sql/hive/src/test/resources/golden/windowing_ntile.q (deterministic)-2-fafa16c0f7697ca28aeb6f2698799562 create mode 100644 sql/hive/src/test/resources/golden/windowing_ntile.q (deterministic)-3-bda0e7c77d6f4712a03389cb5032bc6d create mode 100644 sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 1-0-2e0cbc2d7c5f16657edacd9e7209e6e7 create mode 100644 sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 1-1-5c5f373e325115d710a7a23fcb1626f1 create mode 100644 sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 1-2-ac487cc1b94130bf9ce00e07c7075f65 create mode 100644 sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 1-3-b82dfa24123047be4b4e3c27c3997d34 create mode 100644 sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 2-0-81bb7f49a55385878637c8aac4d08e5 create mode 100644 sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 3-0-58a982694ba2b1e34de82b1de54936a0 create mode 100644 sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 4-0-12cc78f3953c3e6b5411ddc729541bf0 create mode 100644 sql/hive/src/test/resources/golden/windowing_udaf.q (deterministic)-0-6642a21d87e0401ba1a668ea8b244f0c create mode 100644 sql/hive/src/test/resources/golden/windowing_udaf.q (deterministic)-1-2bf20f39e6ffef258858f7943a974e7e create mode 100644 sql/hive/src/test/resources/golden/windowing_udaf.q (deterministic)-2-16239d2b069789ba99fbac50c4f0724f create mode 100644 sql/hive/src/test/resources/golden/windowing_udaf.q (deterministic)-3-d90b27fca067b0b3c48d873b3ef32af7 create mode 100644 sql/hive/src/test/resources/golden/windowing_udaf.q (deterministic)-4-f2e4d659b65a833e9281b6786d3d55c1 create mode 100644 sql/hive/src/test/resources/golden/windowing_udaf2-0-96659fde37d7a38ea15b367b47f59ce2 create mode 100644 sql/hive/src/test/resources/golden/windowing_udaf2-1-b4bdee4908b1cb8e240c549ae5cfe4c0 create mode 100644 sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-0-f498cccf82480be03022d2a36f87651e create mode 100644 sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-1-6378faf36ffd3f61e61cee6c0cb70e6 create mode 100644 sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-2-5f0eab306ea3c22b11ace9b542a7ee56 create mode 100644 sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-3-6f104992e0050576085064815de43194 create mode 100644 sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-4-cd2e3d2344810cb3ba843d4c01c81d7e create mode 100644 sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-5-ee44c5cdc80e1c832b702f9fb76d8145 create mode 100644 sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-6-4d78f7b1d172d20c91f5867bc13a42a0 create mode 100644 sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-7-20fdc99aa046b2c41d9b85ab338c749c create mode 100644 sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-8-45a1d7c2aba45d761e19ff4dfdf5463e create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 5e42b409dcc59..7b543b6c2aa42 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst.analysis +import scala.collection.mutable.ArrayBuffer + import org.apache.spark.util.collection.OpenHashSet import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.expressions._ @@ -61,6 +63,7 @@ class Analyzer( ResolveGenerate :: ImplicitGenerate :: ResolveFunctions :: + ExtractWindowExpressions :: GlobalAggregates :: UnresolvedHavingClauseAttributes :: TrimGroupingAliases :: @@ -529,6 +532,203 @@ class Analyzer( makeGeneratorOutput(p.generator, p.generatorOutput), p.child) } } + + /** + * Extracts [[WindowExpression]]s from the projectList of a [[Project]] operator and + * aggregateExpressions of an [[Aggregate]] operator and creates individual [[Window]] + * operators for every distinct [[WindowSpecDefinition]]. + * + * This rule handles three cases: + * - A [[Project]] having [[WindowExpression]]s in its projectList; + * - An [[Aggregate]] having [[WindowExpression]]s in its aggregateExpressions. + * - An [[Filter]]->[[Aggregate]] pattern representing GROUP BY with a HAVING + * clause and the [[Aggregate]] has [[WindowExpression]]s in its aggregateExpressions. + * Note: If there is a GROUP BY clause in the query, aggregations and corresponding + * filters (expressions in the HAVING clause) should be evaluated before any + * [[WindowExpression]]. If a query has SELECT DISTINCT, the DISTINCT part should be + * evaluated after all [[WindowExpression]]s. + * + * For every case, the transformation works as follows: + * 1. For a list of [[Expression]]s (a projectList or an aggregateExpressions), partitions + * it two lists of [[Expression]]s, one for all [[WindowExpression]]s and another for + * all regular expressions. + * 2. For all [[WindowExpression]]s, groups them based on their [[WindowSpecDefinition]]s. + * 3. For every distinct [[WindowSpecDefinition]], creates a [[Window]] operator and inserts + * it into the plan tree. + */ + object ExtractWindowExpressions extends Rule[LogicalPlan] { + def hasWindowFunction(projectList: Seq[NamedExpression]): Boolean = + projectList.exists(hasWindowFunction) + + def hasWindowFunction(expr: NamedExpression): Boolean = { + expr.find { + case window: WindowExpression => true + case _ => false + }.isDefined + } + + /** + * From a Seq of [[NamedExpression]]s, extract window expressions and + * other regular expressions. + */ + def extract( + expressions: Seq[NamedExpression]): (Seq[NamedExpression], Seq[NamedExpression]) = { + // First, we simple partition the input expressions to two part, one having + // WindowExpressions and another one without WindowExpressions. + val (windowExpressions, regularExpressions) = expressions.partition(hasWindowFunction) + + // Then, we need to extract those regular expressions used in the WindowExpression. + // For example, when we have col1 - Sum(col2 + col3) OVER (PARTITION BY col4 ORDER BY col5), + // we need to make sure that col1 to col5 are all projected from the child of the Window + // operator. + val extractedExprBuffer = new ArrayBuffer[NamedExpression]() + def extractExpr(expr: Expression): Expression = expr match { + case ne: NamedExpression => + // If a named expression is not in regularExpressions, add extract it and replace it + // with an AttributeReference. + val missingExpr = + AttributeSet(Seq(expr)) -- (regularExpressions ++ extractedExprBuffer) + if (missingExpr.nonEmpty) { + extractedExprBuffer += ne + } + ne.toAttribute + case e: Expression if e.foldable => + e // No need to create an attribute reference if it will be evaluated as a Literal. + case e: Expression => + // For other expressions, we extract it and replace it with an AttributeReference (with + // an interal column name, e.g. "_w0"). + val withName = Alias(e, s"_w${extractedExprBuffer.length}")() + extractedExprBuffer += withName + withName.toAttribute + } + + // Now, we extract expressions from windowExpressions by using extractExpr. + val newWindowExpressions = windowExpressions.map { + _.transform { + // Extracts children expressions of a WindowFunction (input parameters of + // a WindowFunction). + case wf : WindowFunction => + val newChildren = wf.children.map(extractExpr(_)) + wf.withNewChildren(newChildren) + + // Extracts expressions from the partition spec and order spec. + case wsc @ WindowSpecDefinition(partitionSpec, orderSpec, _) => + val newPartitionSpec = partitionSpec.map(extractExpr(_)) + val newOrderSpec = orderSpec.map { so => + val newChild = extractExpr(so.child) + so.copy(child = newChild) + } + wsc.copy(partitionSpec = newPartitionSpec, orderSpec = newOrderSpec) + + // Extracts AggregateExpression. For example, for SUM(x) - Sum(y) OVER (...), + // we need to extract SUM(x). + case agg: AggregateExpression => + val withName = Alias(agg, s"_w${extractedExprBuffer.length}")() + extractedExprBuffer += withName + withName.toAttribute + }.asInstanceOf[NamedExpression] + } + + (newWindowExpressions, regularExpressions ++ extractedExprBuffer) + } + + /** + * Adds operators for Window Expressions. Every Window operator handles a single Window Spec. + */ + def addWindow(windowExpressions: Seq[NamedExpression], child: LogicalPlan): LogicalPlan = { + // First, we group window expressions based on their Window Spec. + val groupedWindowExpression = windowExpressions.groupBy { expr => + val windowExpression = expr.find { + case window: WindowExpression => true + case other => false + }.map(_.asInstanceOf[WindowExpression].windowSpec) + windowExpression.getOrElse( + failAnalysis(s"$windowExpressions does not have any WindowExpression.")) + }.toSeq + + // For every Window Spec, we add a Window operator and set currentChild as the child of it. + var currentChild = child + var i = 0 + while (i < groupedWindowExpression.size) { + val (windowSpec, windowExpressions) = groupedWindowExpression(i) + // Set currentChild to the newly created Window operator. + currentChild = Window(currentChild.output, windowExpressions, windowSpec, currentChild) + + // Move to next WindowExpression. + i += 1 + } + + // We return the top operator. + currentChild + } + + // We have to use transformDown at here to make sure the rule of + // "Aggregate with Having clause" will be triggered. + def apply(plan: LogicalPlan): LogicalPlan = plan transformDown { + // Lookup WindowSpecDefinitions. This rule works with unresolved children. + case WithWindowDefinition(windowDefinitions, child) => + child.transform { + case plan => plan.transformExpressions { + case UnresolvedWindowExpression(c, WindowSpecReference(windowName)) => + val errorMessage = + s"Window specification $windowName is not defined in the WINDOW clause." + val windowSpecDefinition = + windowDefinitions + .get(windowName) + .getOrElse(failAnalysis(errorMessage)) + WindowExpression(c, windowSpecDefinition) + } + } + + // Aggregate with Having clause. This rule works with an unresolved Aggregate because + // a resolved Aggregate will not have Window Functions. + case f @ Filter(condition, a @ Aggregate(groupingExprs, aggregateExprs, child)) + if child.resolved && + hasWindowFunction(aggregateExprs) && + !a.expressions.exists(!_.resolved) => + val (windowExpressions, aggregateExpressions) = extract(aggregateExprs) + // Create an Aggregate operator to evaluate aggregation functions. + val withAggregate = Aggregate(groupingExprs, aggregateExpressions, child) + // Add a Filter operator for conditions in the Having clause. + val withFilter = Filter(condition, withAggregate) + val withWindow = addWindow(windowExpressions, withFilter) + + // Finally, generate output columns according to the original projectList. + val finalProjectList = aggregateExprs.map (_.toAttribute) + Project(finalProjectList, withWindow) + + case p: LogicalPlan if !p.childrenResolved => p + + // Aggregate without Having clause. + case a @ Aggregate(groupingExprs, aggregateExprs, child) + if hasWindowFunction(aggregateExprs) && + !a.expressions.exists(!_.resolved) => + val (windowExpressions, aggregateExpressions) = extract(aggregateExprs) + // Create an Aggregate operator to evaluate aggregation functions. + val withAggregate = Aggregate(groupingExprs, aggregateExpressions, child) + // Add Window operators. + val withWindow = addWindow(windowExpressions, withAggregate) + + // Finally, generate output columns according to the original projectList. + val finalProjectList = aggregateExprs.map (_.toAttribute) + Project(finalProjectList, withWindow) + + // We only extract Window Expressions after all expressions of the Project + // have been resolved. + case p @ Project(projectList, child) + if hasWindowFunction(projectList) && !p.expressions.exists(!_.resolved) => + val (windowExpressions, regularExpressions) = extract(projectList) + // We add a project to get all needed expressions for window expressions from the child + // of the original Project operator. + val withProject = Project(regularExpressions, child) + // Add Window operators. + val withWindow = addWindow(windowExpressions, withProject) + + // Finally, generate output columns according to the original projectList. + val finalProjectList = projectList.map (_.toAttribute) + Project(finalProjectList, withWindow) + } + } } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index 2381689e17525..c8288c6767004 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -70,6 +70,11 @@ trait CheckAnalysis { failAnalysis( s"invalid expression ${b.prettyString} " + s"between ${b.left.simpleString} and ${b.right.simpleString}") + + case w @ WindowExpression(windowFunction, windowSpec) if windowSpec.validate.nonEmpty => + // The window spec is not valid. + val reason = windowSpec.validate.get + failAnalysis(s"Window specification $windowSpec is not valid because $reason") } operator match { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala index c2866cd955409..8cae548279eb1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala @@ -548,3 +548,97 @@ class JoinedRow5 extends Row { } } } + +/** + * JIT HACK: Replace with macros + */ +class JoinedRow6 extends Row { + private[this] var row1: Row = _ + private[this] var row2: Row = _ + + def this(left: Row, right: Row) = { + this() + row1 = left + row2 = right + } + + /** Updates this JoinedRow to used point at two new base rows. Returns itself. */ + def apply(r1: Row, r2: Row): Row = { + row1 = r1 + row2 = r2 + this + } + + /** Updates this JoinedRow by updating its left base row. Returns itself. */ + def withLeft(newLeft: Row): Row = { + row1 = newLeft + this + } + + /** Updates this JoinedRow by updating its right base row. Returns itself. */ + def withRight(newRight: Row): Row = { + row2 = newRight + this + } + + override def toSeq: Seq[Any] = row1.toSeq ++ row2.toSeq + + override def length: Int = row1.length + row2.length + + override def apply(i: Int): Any = + if (i < row1.length) row1(i) else row2(i - row1.length) + + override def isNullAt(i: Int): Boolean = + if (i < row1.length) row1.isNullAt(i) else row2.isNullAt(i - row1.length) + + override def getInt(i: Int): Int = + if (i < row1.length) row1.getInt(i) else row2.getInt(i - row1.length) + + override def getLong(i: Int): Long = + if (i < row1.length) row1.getLong(i) else row2.getLong(i - row1.length) + + override def getDouble(i: Int): Double = + if (i < row1.length) row1.getDouble(i) else row2.getDouble(i - row1.length) + + override def getBoolean(i: Int): Boolean = + if (i < row1.length) row1.getBoolean(i) else row2.getBoolean(i - row1.length) + + override def getShort(i: Int): Short = + if (i < row1.length) row1.getShort(i) else row2.getShort(i - row1.length) + + override def getByte(i: Int): Byte = + if (i < row1.length) row1.getByte(i) else row2.getByte(i - row1.length) + + override def getFloat(i: Int): Float = + if (i < row1.length) row1.getFloat(i) else row2.getFloat(i - row1.length) + + override def getString(i: Int): String = + if (i < row1.length) row1.getString(i) else row2.getString(i - row1.length) + + override def getAs[T](i: Int): T = + if (i < row1.length) row1.getAs[T](i) else row2.getAs[T](i - row1.length) + + override def copy(): Row = { + val totalSize = row1.length + row2.length + val copiedValues = new Array[Any](totalSize) + var i = 0 + while(i < totalSize) { + copiedValues(i) = apply(i) + i += 1 + } + new GenericRow(copiedValues) + } + + override def toString: String = { + // Make sure toString never throws NullPointerException. + if ((row1 eq null) && (row2 eq null)) { + "[ empty row ]" + } else if (row1 eq null) { + row2.mkString("[", ",", "]") + } else if (row2 eq null) { + row1.mkString("[", ",", "]") + } else { + mkString("[", ",", "]") + } + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala new file mode 100644 index 0000000000000..099d67ca7fee3 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala @@ -0,0 +1,340 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.expressions + +import org.apache.spark.sql.catalyst.analysis.UnresolvedException +import org.apache.spark.sql.catalyst.errors.TreeNodeException +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.types.{NumericType, DataType} + +/** + * The trait of the Window Specification (specified in the OVER clause or WINDOW clause) for + * Window Functions. + */ +sealed trait WindowSpec + +/** + * The specification for a window function. + * @param partitionSpec It defines the way that input rows are partitioned. + * @param orderSpec It defines the ordering of rows in a partition. + * @param frameSpecification It defines the window frame in a partition. + */ +case class WindowSpecDefinition( + partitionSpec: Seq[Expression], + orderSpec: Seq[SortOrder], + frameSpecification: WindowFrame) extends Expression with WindowSpec { + + def validate: Option[String] = frameSpecification match { + case UnspecifiedFrame => + Some("Found a UnspecifiedFrame. It should be converted to a SpecifiedWindowFrame " + + "during analysis. Please file a bug report.") + case frame: SpecifiedWindowFrame => frame.validate.orElse { + def checkValueBasedBoundaryForRangeFrame(): Option[String] = { + if (orderSpec.length > 1) { + // It is not allowed to have a value-based PRECEDING and FOLLOWING + // as the boundary of a Range Window Frame. + Some("This Range Window Frame only accepts at most one ORDER BY expression.") + } else if (orderSpec.nonEmpty && !orderSpec.head.dataType.isInstanceOf[NumericType]) { + Some("The data type of the expression in the ORDER BY clause should be a numeric type.") + } else { + None + } + } + + (frame.frameType, frame.frameStart, frame.frameEnd) match { + case (RangeFrame, vp: ValuePreceding, _) => checkValueBasedBoundaryForRangeFrame() + case (RangeFrame, vf: ValueFollowing, _) => checkValueBasedBoundaryForRangeFrame() + case (RangeFrame, _, vp: ValuePreceding) => checkValueBasedBoundaryForRangeFrame() + case (RangeFrame, _, vf: ValueFollowing) => checkValueBasedBoundaryForRangeFrame() + case (_, _, _) => None + } + } + } + + type EvaluatedType = Any + + override def children: Seq[Expression] = partitionSpec ++ orderSpec + + override lazy val resolved: Boolean = + childrenResolved && frameSpecification.isInstanceOf[SpecifiedWindowFrame] + + + override def toString: String = simpleString + + override def eval(input: Row): EvaluatedType = throw new UnsupportedOperationException + override def nullable: Boolean = true + override def foldable: Boolean = false + override def dataType: DataType = throw new UnsupportedOperationException +} + +/** + * A Window specification reference that refers to the [[WindowSpecDefinition]] defined + * under the name `name`. + */ +case class WindowSpecReference(name: String) extends WindowSpec + +/** + * The trait used to represent the type of a Window Frame. + */ +sealed trait FrameType + +/** + * RowFrame treats rows in a partition individually. When a [[ValuePreceding]] + * or a [[ValueFollowing]] is used as its [[FrameBoundary]], the value is considered + * as a physical offset. + * For example, `ROW BETWEEN 1 PRECEDING AND 1 FOLLOWING` represents a 3-row frame, + * from the row precedes the current row to the row follows the current row. + */ +case object RowFrame extends FrameType + +/** + * RangeFrame treats rows in a partition as groups of peers. + * All rows having the same `ORDER BY` ordering are considered as peers. + * When a [[ValuePreceding]] or a [[ValueFollowing]] is used as its [[FrameBoundary]], + * the value is considered as a logical offset. + * For example, assuming the value of the current row's `ORDER BY` expression `expr` is `v`, + * `RANGE BETWEEN 1 PRECEDING AND 1 FOLLOWING` represents a frame containing rows whose values + * `expr` are in the range of [v-1, v+1]. + * + * If `ORDER BY` clause is not defined, all rows in the partition is considered as peers + * of the current row. + */ +case object RangeFrame extends FrameType + +/** + * The trait used to represent the type of a Window Frame Boundary. + */ +sealed trait FrameBoundary { + def notFollows(other: FrameBoundary): Boolean +} + +/** UNBOUNDED PRECEDING boundary. */ +case object UnboundedPreceding extends FrameBoundary { + def notFollows(other: FrameBoundary): Boolean = other match { + case UnboundedPreceding => true + case vp: ValuePreceding => true + case CurrentRow => true + case vf: ValueFollowing => true + case UnboundedFollowing => true + } + + override def toString: String = "UNBOUNDED PRECEDING" +} + +/** PRECEDING boundary. */ +case class ValuePreceding(value: Int) extends FrameBoundary { + def notFollows(other: FrameBoundary): Boolean = other match { + case UnboundedPreceding => false + case ValuePreceding(anotherValue) => value >= anotherValue + case CurrentRow => true + case vf: ValueFollowing => true + case UnboundedFollowing => true + } + + override def toString: String = s"$value PRECEDING" +} + +/** CURRENT ROW boundary. */ +case object CurrentRow extends FrameBoundary { + def notFollows(other: FrameBoundary): Boolean = other match { + case UnboundedPreceding => false + case vp: ValuePreceding => false + case CurrentRow => true + case vf: ValueFollowing => true + case UnboundedFollowing => true + } + + override def toString: String = "CURRENT ROW" +} + +/** FOLLOWING boundary. */ +case class ValueFollowing(value: Int) extends FrameBoundary { + def notFollows(other: FrameBoundary): Boolean = other match { + case UnboundedPreceding => false + case vp: ValuePreceding => false + case CurrentRow => false + case ValueFollowing(anotherValue) => value <= anotherValue + case UnboundedFollowing => true + } + + override def toString: String = s"$value FOLLOWING" +} + +/** UNBOUNDED FOLLOWING boundary. */ +case object UnboundedFollowing extends FrameBoundary { + def notFollows(other: FrameBoundary): Boolean = other match { + case UnboundedPreceding => false + case vp: ValuePreceding => false + case CurrentRow => false + case vf: ValueFollowing => false + case UnboundedFollowing => true + } + + override def toString: String = "UNBOUNDED FOLLOWING" +} + +/** + * The trait used to represent the a Window Frame. + */ +sealed trait WindowFrame + +/** Used as a place holder when a frame specification is not defined. */ +case object UnspecifiedFrame extends WindowFrame + +/** A specified Window Frame. */ +case class SpecifiedWindowFrame( + frameType: FrameType, + frameStart: FrameBoundary, + frameEnd: FrameBoundary) extends WindowFrame { + + /** If this WindowFrame is valid or not. */ + def validate: Option[String] = (frameType, frameStart, frameEnd) match { + case (_, UnboundedFollowing, _) => + Some(s"$UnboundedFollowing is not allowed as the start of a Window Frame.") + case (_, _, UnboundedPreceding) => + Some(s"$UnboundedPreceding is not allowed as the end of a Window Frame.") + // case (RowFrame, start, end) => ??? RowFrame specific rule + // case (RangeFrame, start, end) => ??? RangeFrame specific rule + case (_, start, end) => + if (start.notFollows(end)) { + None + } else { + val reason = + s"The end of this Window Frame $end is smaller than the start of " + + s"this Window Frame $start." + Some(reason) + } + } + + override def toString: String = frameType match { + case RowFrame => s"ROWS BETWEEN $frameStart AND $frameEnd" + case RangeFrame => s"RANGE BETWEEN $frameStart AND $frameEnd" + } +} + +object SpecifiedWindowFrame { + /** + * + * @param hasOrderSpecification If the window spec has order by expressions. + * @param acceptWindowFrame If the window function accepts user-specified frame. + * @return + */ + def defaultWindowFrame( + hasOrderSpecification: Boolean, + acceptWindowFrame: Boolean): SpecifiedWindowFrame = { + if (hasOrderSpecification && acceptWindowFrame) { + // If order spec is defined and the window function supports user specified window frames, + // the default frame is RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW. + SpecifiedWindowFrame(RangeFrame, UnboundedPreceding, CurrentRow) + } else { + // Otherwise, the default frame is + // ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING. + SpecifiedWindowFrame(RowFrame, UnboundedPreceding, UnboundedFollowing) + } + } +} + +/** + * Every window function needs to maintain a output buffer for its output. + * It should expect that for a n-row window frame, it will be called n times + * to retrieve value corresponding with these n rows. + */ +trait WindowFunction extends Expression { + self: Product => + + def init(): Unit + + def reset(): Unit + + def prepareInputParameters(input: Row): AnyRef + + def update(input: AnyRef): Unit + + def batchUpdate(inputs: Array[AnyRef]): Unit + + def evaluate(): Unit + + def get(index: Int): Any + + def newInstance(): WindowFunction +} + +case class UnresolvedWindowFunction( + name: String, + children: Seq[Expression]) + extends Expression with WindowFunction { + + override def dataType: DataType = throw new UnresolvedException(this, "dataType") + override def foldable: Boolean = throw new UnresolvedException(this, "foldable") + override def nullable: Boolean = throw new UnresolvedException(this, "nullable") + override lazy val resolved = false + + override def init(): Unit = + throw new UnresolvedException(this, "init") + override def reset(): Unit = + throw new UnresolvedException(this, "reset") + override def prepareInputParameters(input: Row): AnyRef = + throw new UnresolvedException(this, "prepareInputParameters") + override def update(input: AnyRef): Unit = + throw new UnresolvedException(this, "update") + override def batchUpdate(inputs: Array[AnyRef]): Unit = + throw new UnresolvedException(this, "batchUpdate") + override def evaluate(): Unit = + throw new UnresolvedException(this, "evaluate") + override def get(index: Int): Any = + throw new UnresolvedException(this, "get") + // Unresolved functions are transient at compile time and don't get evaluated during execution. + override def eval(input: Row = null): EvaluatedType = + throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}") + + override def toString: String = s"'$name(${children.mkString(",")})" + + override def newInstance(): WindowFunction = + throw new UnresolvedException(this, "newInstance") +} + +case class UnresolvedWindowExpression( + child: UnresolvedWindowFunction, + windowSpec: WindowSpecReference) extends UnaryExpression { + override def dataType: DataType = throw new UnresolvedException(this, "dataType") + override def foldable: Boolean = throw new UnresolvedException(this, "foldable") + override def nullable: Boolean = throw new UnresolvedException(this, "nullable") + override lazy val resolved = false + + // Unresolved functions are transient at compile time and don't get evaluated during execution. + override def eval(input: Row = null): EvaluatedType = + throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}") +} + +case class WindowExpression( + windowFunction: WindowFunction, + windowSpec: WindowSpecDefinition) extends Expression { + override type EvaluatedType = Any + + override def children: Seq[Expression] = + windowFunction :: windowSpec :: Nil + + override def eval(input: Row): EvaluatedType = + throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}") + + override def dataType: DataType = windowFunction.dataType + override def foldable: Boolean = windowFunction.foldable + override def nullable: Boolean = windowFunction.nullable + + override def toString: String = s"$windowFunction $windowSpec" +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala index 21208c8a5c281..ba0abb2df596c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala @@ -25,13 +25,14 @@ case class Project(projectList: Seq[NamedExpression], child: LogicalPlan) extend override def output: Seq[Attribute] = projectList.map(_.toAttribute) override lazy val resolved: Boolean = { - val containsAggregatesOrGenerators = projectList.exists ( _.collect { + val hasSpecialExpressions = projectList.exists ( _.collect { case agg: AggregateExpression => agg case generator: Generator => generator + case window: WindowExpression => window }.nonEmpty ) - !expressions.exists(!_.resolved) && childrenResolved && !containsAggregatesOrGenerators + !expressions.exists(!_.resolved) && childrenResolved && !hasSpecialExpressions } } @@ -170,6 +171,12 @@ case class With(child: LogicalPlan, cteRelations: Map[String, Subquery]) extends override def output: Seq[Attribute] = child.output } +case class WithWindowDefinition( + windowDefinitions: Map[String, WindowSpecDefinition], + child: LogicalPlan) extends UnaryNode { + override def output: Seq[Attribute] = child.output +} + case class WriteToFile( path: String, child: LogicalPlan) extends UnaryNode { @@ -195,9 +202,28 @@ case class Aggregate( child: LogicalPlan) extends UnaryNode { + override lazy val resolved: Boolean = { + val hasWindowExpressions = aggregateExpressions.exists ( _.collect { + case window: WindowExpression => window + }.nonEmpty + ) + + !expressions.exists(!_.resolved) && childrenResolved && !hasWindowExpressions + } + override def output: Seq[Attribute] = aggregateExpressions.map(_.toAttribute) } +case class Window( + projectList: Seq[Attribute], + windowExpressions: Seq[NamedExpression], + windowSpec: WindowSpecDefinition, + child: LogicalPlan) extends UnaryNode { + + override def output: Seq[Attribute] = + (projectList ++ windowExpressions).map(_.toAttribute) +} + /** * Apply the all of the GroupExpressions to every input row, hence we will get * multiple output rows for a input row. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala index 97502ed3afe72..4b93f7d31b808 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala @@ -71,6 +71,15 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { this.eq(other) || this == other } + /** + * Find the first [[TreeNode]] that satisfies the condition specified by `f`. + * The condition is recursively applied to this node and all of its children (pre-order). + */ + def find(f: BaseType => Boolean): Option[BaseType] = f(this) match { + case true => Some(this) + case false => children.foldLeft(None: Option[BaseType]) { (l, r) => l.orElse(r.find(f)) } + } + /** * Runs the given function on this node and then recursively on [[children]]. * @param f the function to be applied to each node in the tree. @@ -151,6 +160,20 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { val remainingNewChildren = newChildren.toBuffer val remainingOldChildren = children.toBuffer val newArgs = productIterator.map { + // This rule is used to handle children is a input argument. + case s: Seq[_] => s.map { + case arg: TreeNode[_] if children contains arg => + val newChild = remainingNewChildren.remove(0) + val oldChild = remainingOldChildren.remove(0) + if (newChild fastEquals oldChild) { + oldChild + } else { + changed = true + newChild + } + case nonChild: AnyRef => nonChild + case null => null + } case arg: TreeNode[_] if children contains arg => val newChild = remainingNewChildren.remove(0) val oldChild = remainingOldChildren.remove(0) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala index 6b393327cc97a..786ddba403f2c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala @@ -22,7 +22,7 @@ import scala.collection.mutable.ArrayBuffer import org.scalatest.FunSuite import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.types.{StringType, NullType} +import org.apache.spark.sql.types.{IntegerType, StringType, NullType} case class Dummy(optKey: Option[Expression]) extends Expression { def children: Seq[Expression] = optKey.toSeq @@ -129,5 +129,47 @@ class TreeNodeSuite extends FunSuite { assert(expected === actual) } + test("find") { + val expression = Add(Literal(1), Multiply(Literal(2), Subtract(Literal(3), Literal(4)))) + // Find the top node. + var actual: Option[Expression] = expression.find { + case add: Add => true + case other => false + } + var expected: Option[Expression] = + Some(Add(Literal(1), Multiply(Literal(2), Subtract(Literal(3), Literal(4))))) + assert(expected === actual) + + // Find the first children. + actual = expression.find { + case Literal(1, IntegerType) => true + case other => false + } + expected = Some(Literal(1)) + assert(expected === actual) + // Find an internal node (Subtract). + actual = expression.find { + case sub: Subtract => true + case other => false + } + expected = Some(Subtract(Literal(3), Literal(4))) + assert(expected === actual) + + // Find a leaf node. + actual = expression.find { + case Literal(3, IntegerType) => true + case other => false + } + expected = Some(Literal(3)) + assert(expected === actual) + + // Find nothing. + actual = expression.find { + case Literal(100, IntegerType) => true + case other => false + } + expected = None + assert(expected === actual) + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 326e8ce4ca524..56a4689eb58f0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -303,6 +303,8 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { execution.Expand(projections, output, planLater(child)) :: Nil case logical.Aggregate(group, agg, child) => execution.Aggregate(partial = false, group, agg, planLater(child)) :: Nil + case logical.Window(projectList, windowExpressions, spec, child) => + execution.Window(projectList, windowExpressions, spec, planLater(child)) :: Nil case logical.Sample(lb, ub, withReplacement, seed, child) => execution.Sample(lb, ub, withReplacement, seed, planLater(child)) :: Nil case logical.LocalRelation(output, data) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Window.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Window.scala new file mode 100644 index 0000000000000..217b559def512 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Window.scala @@ -0,0 +1,480 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution + +import java.util + +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.physical.{AllTuples, Distribution, ClusteredDistribution, Partitioning} +import org.apache.spark.util.collection.CompactBuffer + +/** + * :: DeveloperApi :: + * For every row, evaluates `windowExpression` containing Window Functions and attaches + * the results with other regular expressions (presented by `projectList`). + * Evert operator handles a single Window Specification, `windowSpec`. + */ +case class Window( + projectList: Seq[Attribute], + windowExpression: Seq[NamedExpression], + windowSpec: WindowSpecDefinition, + child: SparkPlan) + extends UnaryNode { + + override def output: Seq[Attribute] = + (projectList ++ windowExpression).map(_.toAttribute) + + override def requiredChildDistribution: Seq[Distribution] = + if (windowSpec.partitionSpec.isEmpty) { + // This operator will be very expensive. + AllTuples :: Nil + } else { + ClusteredDistribution(windowSpec.partitionSpec) :: Nil + } + + // Since window functions are adding columns to the input rows, the child's outputPartitioning + // is preserved. + override def outputPartitioning: Partitioning = child.outputPartitioning + + override def requiredChildOrdering: Seq[Seq[SortOrder]] = { + // The required child ordering has two parts. + // The first part is the expressions in the partition specification. + // We add these expressions to the required ordering to make sure input rows are grouped + // based on the partition specification. So, we only need to process a single partition + // at a time. + // The second part is the expressions specified in the ORDER BY cluase. + // Basically, we first use sort to group rows based on partition specifications and then sort + // Rows in a group based on the order specification. + (windowSpec.partitionSpec.map(SortOrder(_, Ascending)) ++ windowSpec.orderSpec) :: Nil + } + + // Since window functions basically add columns to input rows, this operator + // will not change the ordering of input rows. + override def outputOrdering: Seq[SortOrder] = child.outputOrdering + + case class ComputedWindow( + unbound: WindowExpression, + windowFunction: WindowFunction, + resultAttribute: AttributeReference) + + // A list of window functions that need to be computed for each group. + private[this] val computedWindowExpressions = windowExpression.flatMap { window => + window.collect { + case w: WindowExpression => + ComputedWindow( + w, + BindReferences.bindReference(w.windowFunction, child.output), + AttributeReference(s"windowResult:$w", w.dataType, w.nullable)()) + } + }.toArray + + private[this] val windowFrame = + windowSpec.frameSpecification.asInstanceOf[SpecifiedWindowFrame] + + // Create window functions. + private[this] def windowFunctions(): Array[WindowFunction] = { + val functions = new Array[WindowFunction](computedWindowExpressions.length) + var i = 0 + while (i < computedWindowExpressions.length) { + functions(i) = computedWindowExpressions(i).windowFunction.newInstance() + functions(i).init() + i += 1 + } + functions + } + + // The schema of the result of all window function evaluations + private[this] val computedSchema = computedWindowExpressions.map(_.resultAttribute) + + private[this] val computedResultMap = + computedWindowExpressions.map { w => w.unbound -> w.resultAttribute }.toMap + + private[this] val windowExpressionResult = windowExpression.map { window => + window.transform { + case w: WindowExpression if computedResultMap.contains(w) => computedResultMap(w) + } + } + + def execute(): RDD[Row] = { + child.execute().mapPartitions { iter => + new Iterator[Row] { + + // Although input rows are grouped based on windowSpec.partitionSpec, we need to + // know when we have a new partition. + // This is to manually construct an ordering that can be used to compare rows. + // TODO: We may want to have a newOrdering that takes BoundReferences. + // So, we can take advantave of code gen. + private val partitionOrdering: Ordering[Row] = + RowOrdering.forSchema(windowSpec.partitionSpec.map(_.dataType)) + + // This is used to project expressions for the partition specification. + protected val partitionGenerator = + newMutableProjection(windowSpec.partitionSpec, child.output)() + + // This is ued to project expressions for the order specification. + protected val rowOrderGenerator = + newMutableProjection(windowSpec.orderSpec.map(_.child), child.output)() + + // The position of next output row in the inputRowBuffer. + var rowPosition: Int = 0 + // The number of buffered rows in the inputRowBuffer (the size of the current partition). + var partitionSize: Int = 0 + // The buffer used to buffer rows in a partition. + var inputRowBuffer: CompactBuffer[Row] = _ + // The partition key of the current partition. + var currentPartitionKey: Row = _ + // The partition key of next partition. + var nextPartitionKey: Row = _ + // The first row of next partition. + var firstRowInNextPartition: Row = _ + // Indicates if this partition is the last one in the iter. + var lastPartition: Boolean = false + + def createBoundaryEvaluator(): () => Unit = { + def findPhysicalBoundary( + boundary: FrameBoundary): () => Int = boundary match { + case UnboundedPreceding => () => 0 + case UnboundedFollowing => () => partitionSize - 1 + case CurrentRow => () => rowPosition + case ValuePreceding(value) => + () => + val newPosition = rowPosition - value + if (newPosition > 0) newPosition else 0 + case ValueFollowing(value) => + () => + val newPosition = rowPosition + value + if (newPosition < partitionSize) newPosition else partitionSize - 1 + } + + def findLogicalBoundary( + boundary: FrameBoundary, + searchDirection: Int, + evaluator: Expression, + joinedRow: JoinedRow): () => Int = boundary match { + case UnboundedPreceding => () => 0 + case UnboundedFollowing => () => partitionSize - 1 + case other => + () => { + // CurrentRow, ValuePreceding, or ValueFollowing. + var newPosition = rowPosition + searchDirection + var stopSearch = false + // rowOrderGenerator is a mutable projection. + // We need to make a copy of the returned by rowOrderGenerator since we will + // compare searched row with this currentOrderByValue. + val currentOrderByValue = rowOrderGenerator(inputRowBuffer(rowPosition)).copy() + while (newPosition >= 0 && newPosition < partitionSize && !stopSearch) { + val r = rowOrderGenerator(inputRowBuffer(newPosition)) + stopSearch = + !(evaluator.eval(joinedRow(currentOrderByValue, r)).asInstanceOf[Boolean]) + if (!stopSearch) { + newPosition += searchDirection + } + } + newPosition -= searchDirection + + if (newPosition < 0) { + 0 + } else if (newPosition >= partitionSize) { + partitionSize - 1 + } else { + newPosition + } + } + } + + windowFrame.frameType match { + case RowFrame => + val findStart = findPhysicalBoundary(windowFrame.frameStart) + val findEnd = findPhysicalBoundary(windowFrame.frameEnd) + () => { + frameStart = findStart() + frameEnd = findEnd() + } + case RangeFrame => + val joinedRowForBoundaryEvaluation: JoinedRow = new JoinedRow() + val orderByExpr = windowSpec.orderSpec.head + val currentRowExpr = + BoundReference(0, orderByExpr.dataType, orderByExpr.nullable) + val examedRowExpr = + BoundReference(1, orderByExpr.dataType, orderByExpr.nullable) + val differenceExpr = Abs(Subtract(currentRowExpr, examedRowExpr)) + + val frameStartEvaluator = windowFrame.frameStart match { + case CurrentRow => EqualTo(currentRowExpr, examedRowExpr) + case ValuePreceding(value) => + LessThanOrEqual(differenceExpr, Cast(Literal(value), orderByExpr.dataType)) + case ValueFollowing(value) => + GreaterThanOrEqual(differenceExpr, Cast(Literal(value), orderByExpr.dataType)) + case o => Literal(true) // This is just a dummy expression, we will not use it. + } + + val frameEndEvaluator = windowFrame.frameEnd match { + case CurrentRow => EqualTo(currentRowExpr, examedRowExpr) + case ValuePreceding(value) => + GreaterThanOrEqual(differenceExpr, Cast(Literal(value), orderByExpr.dataType)) + case ValueFollowing(value) => + LessThanOrEqual(differenceExpr, Cast(Literal(value), orderByExpr.dataType)) + case o => Literal(true) // This is just a dummy expression, we will not use it. + } + + val findStart = + findLogicalBoundary( + boundary = windowFrame.frameStart, + searchDirection = -1, + evaluator = frameStartEvaluator, + joinedRow = joinedRowForBoundaryEvaluation) + val findEnd = + findLogicalBoundary( + boundary = windowFrame.frameEnd, + searchDirection = 1, + evaluator = frameEndEvaluator, + joinedRow = joinedRowForBoundaryEvaluation) + () => { + frameStart = findStart() + frameEnd = findEnd() + } + } + } + + val boundaryEvaluator = createBoundaryEvaluator() + // Indicates if we the specified window frame requires us to maintain a sliding frame + // (e.g. RANGES BETWEEN 1 PRECEDING AND CURRENT ROW) or the window frame + // is the entire partition (e.g. ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING). + val requireUpdateFrame: Boolean = { + def requireUpdateBoundary(boundary: FrameBoundary): Boolean = boundary match { + case UnboundedPreceding => false + case UnboundedFollowing => false + case _ => true + } + + requireUpdateBoundary(windowFrame.frameStart) || + requireUpdateBoundary(windowFrame.frameEnd) + } + // The start position of the current frame in the partition. + var frameStart: Int = 0 + // The end position of the current frame in the partition. + var frameEnd: Int = -1 + // Window functions. + val functions: Array[WindowFunction] = windowFunctions() + // Buffers used to store input parameters for window functions. Because we may need to + // maintain a sliding frame, we use this buffer to avoid evaluate the parameters from + // the same row multiple times. + val windowFunctionParameterBuffers: Array[util.LinkedList[AnyRef]] = + functions.map(_ => new util.LinkedList[AnyRef]()) + + // The projection used to generate the final result rows of this operator. + private[this] val resultProjection = + newMutableProjection( + projectList ++ windowExpressionResult, + projectList ++ computedSchema)() + + // The row used to hold results of window functions. + private[this] val windowExpressionResultRow = + new GenericMutableRow(computedSchema.length) + + private[this] val joinedRow = new JoinedRow6 + + // Initialize this iterator. + initialize() + + private def initialize(): Unit = { + if (iter.hasNext) { + val currentRow = iter.next().copy() + // partitionGenerator is a mutable projection. Since we need to track nextPartitionKey, + // we are making a copy of the returned partitionKey at here. + nextPartitionKey = partitionGenerator(currentRow).copy() + firstRowInNextPartition = currentRow + fetchNextPartition() + } else { + // The iter is an empty one. So, we set all of the following variables + // to make sure hasNext will return false. + lastPartition = true + rowPosition = 0 + partitionSize = 0 + } + } + + // Indicates if we will have new output row. + override final def hasNext: Boolean = { + !lastPartition || (rowPosition < partitionSize) + } + + override final def next(): Row = { + if (hasNext) { + if (rowPosition == partitionSize) { + // All rows of this buffer have been consumed. + // We will move to next partition. + fetchNextPartition() + } + // Get the input row for the current output row. + val inputRow = inputRowBuffer(rowPosition) + // Get all results of the window functions for this output row. + var i = 0 + while (i < functions.length) { + windowExpressionResultRow.update(i, functions(i).get(rowPosition)) + i += 1 + } + + // Construct the output row. + val outputRow = resultProjection(joinedRow(inputRow, windowExpressionResultRow)) + // We will move to the next one. + rowPosition += 1 + if (requireUpdateFrame && rowPosition < partitionSize) { + // If we need to maintain a sliding frame and + // we will still work on this partition when next is called next time, do the update. + updateFrame() + } + + // Return the output row. + outputRow + } else { + // no more result + throw new NoSuchElementException + } + } + + // Fetch the next partition. + private def fetchNextPartition(): Unit = { + // Create a new buffer for input rows. + inputRowBuffer = new CompactBuffer[Row]() + // We already have the first row for this partition + // (recorded in firstRowInNextPartition). Add it back. + inputRowBuffer += firstRowInNextPartition + // Set the current partition key. + currentPartitionKey = nextPartitionKey + // Now, we will start to find all rows belonging to this partition. + // Create a variable to track if we see the next partition. + var findNextPartition = false + // The search will stop when we see the next partition or there is no + // input row left in the iter. + while (iter.hasNext && !findNextPartition) { + // Make a copy of the input row since we will put it in the buffer. + val currentRow = iter.next().copy() + // Get the partition key based on the partition specification. + // For the below compare method, we do not need to make a copy of partitionKey. + val partitionKey = partitionGenerator(currentRow) + // Check if the current row belongs the current input row. + val comparing = partitionOrdering.compare(currentPartitionKey, partitionKey) + if (comparing == 0) { + // This row is still in the current partition. + inputRowBuffer += currentRow + } else { + // The current input row is in a different partition. + findNextPartition = true + // partitionGenerator is a mutable projection. + // Since we need to track nextPartitionKey and we determine that it should be set + // as partitionKey, we are making a copy of the partitionKey at here. + nextPartitionKey = partitionKey.copy() + firstRowInNextPartition = currentRow + } + } + + // We have not seen a new partition. It means that there is no new row in the + // iter. The current partition is the last partition of the iter. + if (!findNextPartition) { + lastPartition = true + } + + // We have got all rows for the current partition. + // Set rowPosition to 0 (the next output row will be based on the first + // input row of this partition). + rowPosition = 0 + // The size of this partition. + partitionSize = inputRowBuffer.size + // Reset all parameter buffers of window functions. + var i = 0 + while (i < windowFunctionParameterBuffers.length) { + windowFunctionParameterBuffers(i).clear() + i += 1 + } + frameStart = 0 + frameEnd = -1 + // Create the first window frame for this partition. + // If we do not need to maintain a sliding frame, this frame will + // have the entire partition. + updateFrame() + } + + /** The function used to maintain the sliding frame. */ + private def updateFrame(): Unit = { + // Based on the difference between the new frame and old frame, + // updates the buffers holding input parameters of window functions. + // We will start to prepare input parameters starting from the row + // indicated by offset in the input row buffer. + def updateWindowFunctionParameterBuffers( + numToRemove: Int, + numToAdd: Int, + offset: Int): Unit = { + // First, remove unneeded entries from the head of every buffer. + var i = 0 + while (i < numToRemove) { + var j = 0 + while (j < windowFunctionParameterBuffers.length) { + windowFunctionParameterBuffers(j).remove() + j += 1 + } + i += 1 + } + // Then, add needed entries to the tail of every buffer. + i = 0 + while (i < numToAdd) { + var j = 0 + while (j < windowFunctionParameterBuffers.length) { + // Ask the function to prepare the input parameters. + val parameters = functions(j).prepareInputParameters(inputRowBuffer(i + offset)) + windowFunctionParameterBuffers(j).add(parameters) + j += 1 + } + i += 1 + } + } + + // Record the current frame start point and end point before + // we update them. + val previousFrameStart = frameStart + val previousFrameEnd = frameEnd + boundaryEvaluator() + updateWindowFunctionParameterBuffers( + frameStart - previousFrameStart, + frameEnd - previousFrameEnd, + previousFrameEnd + 1) + // Evaluate the current frame. + evaluateCurrentFrame() + } + + /** Evaluate the current window frame. */ + private def evaluateCurrentFrame(): Unit = { + var i = 0 + while (i < functions.length) { + // Reset the state of the window function. + functions(i).reset() + // Get all buffered input parameters based on rows of this window frame. + val inputParameters = windowFunctionParameterBuffers(i).toArray() + // Send these input parameters to the window function. + functions(i).batchUpdate(inputParameters) + // Ask the function to evaluate based on this window frame. + functions(i).evaluate() + i += 1 + } + } + } + } + } +} diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index 81ee48ef4152f..5e411c2fdba9d 100644 --- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -185,7 +185,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { // Hive does not support buckets. ".*bucket.*", - // No window support yet + // We have our own tests based on these query files. ".*window.*", // Fails in hive with authorization errors. diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index 1d8d0b5c322ad..f25723e53f07c 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -252,6 +252,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { catalog.CreateTables :: catalog.PreInsertionCasts :: ExtractPythonUdfs :: + ResolveHiveWindowFunction :: sources.PreInsertCastAndRename :: Nil } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index 63a8c05f775b8..8a0686a2d81c9 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -421,16 +421,16 @@ private[hive] object HiveQl { } /** - * SELECT MAX(value) FROM src GROUP BY k1, k2, k3 GROUPING SETS((k1, k2), (k2)) - * is equivalent to + * SELECT MAX(value) FROM src GROUP BY k1, k2, k3 GROUPING SETS((k1, k2), (k2)) + * is equivalent to * SELECT MAX(value) FROM src GROUP BY k1, k2 UNION SELECT MAX(value) FROM src GROUP BY k2 * Check the following link for details. - * + * https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C+Grouping+and+Rollup * * The bitmask denotes the grouping expressions validity for a grouping set, * the bitmask also be called as grouping id (`GROUPING__ID`, the virtual column in Hive) - * e.g. In superset (k1, k2, k3), (bit 0: k1, bit 1: k2, and bit 2: k3), the grouping id of + * e.g. In superset (k1, k2, k3), (bit 0: k1, bit 1: k2, and bit 2: k3), the grouping id of * GROUPING SETS (k1, k2) and (k2) should be 3 and 2 respectively. */ protected def extractGroupingSet(children: Seq[ASTNode]): (Seq[Expression], Seq[Int]) = { @@ -444,7 +444,7 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C val bitmasks: Seq[Int] = setASTs.map(set => set match { case Token("TOK_GROUPING_SETS_EXPRESSION", null) => 0 - case Token("TOK_GROUPING_SETS_EXPRESSION", children) => + case Token("TOK_GROUPING_SETS_EXPRESSION", children) => children.foldLeft(0)((bitmap, col) => { val colString = col.asInstanceOf[ASTNode].toStringTree() require(keyMap.contains(colString), s"$colString doens't show up in the GROUP BY list") @@ -613,7 +613,8 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C clusterByClause :: distributeByClause :: limitClause :: - lateralViewClause :: Nil) = { + lateralViewClause :: + windowClause :: Nil) = { getClauses( Seq( "TOK_INSERT_INTO", @@ -631,15 +632,16 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C "TOK_CLUSTERBY", "TOK_DISTRIBUTEBY", "TOK_LIMIT", - "TOK_LATERAL_VIEW"), + "TOK_LATERAL_VIEW", + "WINDOW"), singleInsert) } - + val relations = fromClause match { case Some(f) => nodeToRelation(f) case None => OneRowRelation } - + val withWhere = whereClause.map { whereNode => val Seq(whereExpr) = whereNode.getChildren.toSeq Filter(nodeToExpr(whereExpr), relations) @@ -691,7 +693,7 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C val serdeProps = propsClause.map { case Token("TOK_TABLEPROPERTY", Token(name, Nil) :: Token(value, Nil) :: Nil) => (name, value) - } + } (Nil, serdeClass, serdeProps) case Nil => (Nil, "", Nil) @@ -736,7 +738,7 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C // The projection of the query can either be a normal projection, an aggregation // (if there is a group by) or a script transformation. val withProject: LogicalPlan = transformation.getOrElse { - val selectExpressions = + val selectExpressions = nameExpressions(select.getChildren.flatMap(selExprNodeToExpr).toSeq) Seq( groupByClause.map(e => e match { @@ -764,31 +766,34 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C Some(Project(selectExpressions, withLateralView))).flatten.head } - val withDistinct = - if (selectDistinctClause.isDefined) Distinct(withProject) else withProject - + // Handle HAVING clause. val withHaving = havingClause.map { h => val havingExpr = h.getChildren.toSeq match { case Seq(hexpr) => nodeToExpr(hexpr) } // Note that we added a cast to boolean. If the expression itself is already boolean, // the optimizer will get rid of the unnecessary cast. - Filter(Cast(havingExpr, BooleanType), withDistinct) - }.getOrElse(withDistinct) + Filter(Cast(havingExpr, BooleanType), withProject) + }.getOrElse(withProject) + + // Handle SELECT DISTINCT + val withDistinct = + if (selectDistinctClause.isDefined) Distinct(withHaving) else withHaving + // Handle ORDER BY, SORT BY, DISTRIBETU BY, and CLUSTER BY clause. val withSort = (orderByClause, sortByClause, distributeByClause, clusterByClause) match { case (Some(totalOrdering), None, None, None) => - Sort(totalOrdering.getChildren.map(nodeToSortOrder), true, withHaving) + Sort(totalOrdering.getChildren.map(nodeToSortOrder), true, withDistinct) case (None, Some(perPartitionOrdering), None, None) => - Sort(perPartitionOrdering.getChildren.map(nodeToSortOrder), false, withHaving) + Sort(perPartitionOrdering.getChildren.map(nodeToSortOrder), false, withDistinct) case (None, None, Some(partitionExprs), None) => - RepartitionByExpression(partitionExprs.getChildren.map(nodeToExpr), withHaving) + RepartitionByExpression(partitionExprs.getChildren.map(nodeToExpr), withDistinct) case (None, Some(perPartitionOrdering), Some(partitionExprs), None) => Sort(perPartitionOrdering.getChildren.map(nodeToSortOrder), false, - RepartitionByExpression(partitionExprs.getChildren.map(nodeToExpr), withHaving)) + RepartitionByExpression(partitionExprs.getChildren.map(nodeToExpr), withDistinct)) case (None, None, None, Some(clusterExprs)) => Sort(clusterExprs.getChildren.map(nodeToExpr).map(SortOrder(_, Ascending)), false, - RepartitionByExpression(clusterExprs.getChildren.map(nodeToExpr), withHaving)) - case (None, None, None, None) => withHaving + RepartitionByExpression(clusterExprs.getChildren.map(nodeToExpr), withDistinct)) + case (None, None, None, None) => withDistinct case _ => sys.error("Unsupported set of ordering / distribution clauses.") } @@ -797,6 +802,27 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C .map(Limit(_, withSort)) .getOrElse(withSort) + // Collect all window specifications defined in the WINDOW clause. + val windowDefinitions = windowClause.map(_.getChildren.toSeq.collect { + case Token("TOK_WINDOWDEF", + Token(windowName, Nil) :: Token("TOK_WINDOWSPEC", spec) :: Nil) => + windowName -> nodesToWindowSpecification(spec) + }.toMap) + // Handle cases like + // window w1 as (partition by p_mfgr order by p_name + // range between 2 preceding and 2 following), + // w2 as w1 + val resolvedCrossReference = windowDefinitions.map { + windowDefMap => windowDefMap.map { + case (windowName, WindowSpecReference(other)) => + (windowName, windowDefMap(other).asInstanceOf[WindowSpecDefinition]) + case o => o.asInstanceOf[(String, WindowSpecDefinition)] + } + } + + val withWindowDefinitions = + resolvedCrossReference.map(WithWindowDefinition(_, withLimit)).getOrElse(withLimit) + // TOK_INSERT_INTO means to add files to the table. // TOK_DESTINATION means to overwrite the table. val resultDestination = @@ -804,7 +830,7 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C val overwrite = intoClause.isEmpty nodeToDest( resultDestination, - withLimit, + withWindowDefinitions, overwrite) } @@ -1053,7 +1079,6 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C throw new NotImplementedError(s"No parse rules for:\n ${dumpTree(a).toString} ") } - protected val escapedIdentifier = "`([^`]+)`".r /** Strips backticks from ident if present */ protected def cleanIdentifier(ident: String): String = ident match { @@ -1250,6 +1275,25 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C Substring(nodeToExpr(string), nodeToExpr(pos), nodeToExpr(length)) case Token("TOK_FUNCTION", Token(COALESCE(), Nil) :: list) => Coalesce(list.map(nodeToExpr)) + /* Window Functions */ + case Token("TOK_FUNCTION", Token(name, Nil) +: args :+ Token("TOK_WINDOWSPEC", spec)) => + val function = UnresolvedWindowFunction(name, args.map(nodeToExpr)) + nodesToWindowSpecification(spec) match { + case reference: WindowSpecReference => + UnresolvedWindowExpression(function, reference) + case definition: WindowSpecDefinition => + WindowExpression(function, definition) + } + case Token("TOK_FUNCTIONSTAR", Token(name, Nil) :: Token("TOK_WINDOWSPEC", spec) :: Nil) => + // Safe to use Literal(1)? + val function = UnresolvedWindowFunction(name, Literal(1) :: Nil) + nodesToWindowSpecification(spec) match { + case reference: WindowSpecReference => + UnresolvedWindowExpression(function, reference) + case definition: WindowSpecDefinition => + WindowExpression(function, definition) + } + /* UDFs - Must be last otherwise will preempt built in functions */ case Token("TOK_FUNCTION", Token(name, Nil) :: args) => UnresolvedFunction(name, args.map(nodeToExpr)) @@ -1312,6 +1356,89 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C """.stripMargin) } + def nodesToWindowSpecification(nodes: Seq[ASTNode]): WindowSpec = nodes match { + case Token(windowName, Nil) :: Nil => + // Refer to a window spec defined in the window clause. + WindowSpecReference(windowName) + case Nil => + // OVER() + WindowSpecDefinition( + partitionSpec = Nil, + orderSpec = Nil, + frameSpecification = UnspecifiedFrame) + case spec => + val (partitionClause :: rowFrame :: rangeFrame :: Nil) = + getClauses( + Seq( + "TOK_PARTITIONINGSPEC", + "TOK_WINDOWRANGE", + "TOK_WINDOWVALUES"), + spec) + + // Handle Partition By and Order By. + val (partitionSpec, orderSpec) = partitionClause.map { partitionAndOrdering => + val (partitionByClause :: orderByClause :: sortByClause :: clusterByClause :: Nil) = + getClauses( + Seq("TOK_DISTRIBUTEBY", "TOK_ORDERBY", "TOK_SORTBY", "TOK_CLUSTERBY"), + partitionAndOrdering.getChildren.toSeq.asInstanceOf[Seq[ASTNode]]) + + (partitionByClause, orderByClause.orElse(sortByClause), clusterByClause) match { + case (Some(partitionByExpr), Some(orderByExpr), None) => + (partitionByExpr.getChildren.map(nodeToExpr), + orderByExpr.getChildren.map(nodeToSortOrder)) + case (Some(partitionByExpr), None, None) => + (partitionByExpr.getChildren.map(nodeToExpr), Nil) + case (None, Some(orderByExpr), None) => + (Nil, orderByExpr.getChildren.map(nodeToSortOrder)) + case (None, None, Some(clusterByExpr)) => + val expressions = clusterByExpr.getChildren.map(nodeToExpr) + (expressions, expressions.map(SortOrder(_, Ascending))) + case _ => + throw new NotImplementedError( + s"""No parse rules for Node ${partitionAndOrdering.getName} + """.stripMargin) + } + }.getOrElse { + (Nil, Nil) + } + + // Handle Window Frame + val windowFrame = + if (rowFrame.isEmpty && rangeFrame.isEmpty) { + UnspecifiedFrame + } else { + val frameType = rowFrame.map(_ => RowFrame).getOrElse(RangeFrame) + def nodeToBoundary(node: Node): FrameBoundary = node match { + case Token("preceding", Token(count, Nil) :: Nil) => + if (count == "unbounded") UnboundedPreceding else ValuePreceding(count.toInt) + case Token("following", Token(count, Nil) :: Nil) => + if (count == "unbounded") UnboundedFollowing else ValueFollowing(count.toInt) + case Token("current", Nil) => CurrentRow + case _ => + throw new NotImplementedError( + s"""No parse rules for the Window Frame Boundary based on Node ${node.getName} + """.stripMargin) + } + + rowFrame.orElse(rangeFrame).map { frame => + frame.getChildren.toList match { + case precedingNode :: followingNode :: Nil => + SpecifiedWindowFrame( + frameType, + nodeToBoundary(precedingNode), + nodeToBoundary(followingNode)) + case precedingNode :: Nil => + SpecifiedWindowFrame(frameType, nodeToBoundary(precedingNode), CurrentRow) + case _ => + throw new NotImplementedError( + s"""No parse rules for the Window Frame based on Node ${frame.getName} + """.stripMargin) + } + }.getOrElse(sys.error(s"If you see this, please file a bug report with your query.")) + } + + WindowSpecDefinition(partitionSpec, orderSpec, windowFrame) + } val explode = "(?i)explode".r def nodesToGenerator(nodes: Seq[Node]): (Generator, Seq[String]) = { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala index 4b6f0ad75f54f..fd0b6f058595d 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala @@ -17,27 +17,27 @@ package org.apache.spark.sql.hive +import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator.AggregationBuffer import org.apache.hadoop.hive.ql.udf.generic.GenericUDFUtils.ConversionHelper +import org.apache.spark.sql.AnalysisException import scala.collection.mutable.ArrayBuffer import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspector, ConstantObjectInspector} import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory.ObjectInspectorOptions import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory -import org.apache.hadoop.hive.ql.exec.{UDF, UDAF} -import org.apache.hadoop.hive.ql.exec.{FunctionInfo, FunctionRegistry} +import org.apache.hadoop.hive.ql.exec._ import org.apache.hadoop.hive.ql.udf.{UDFType => HiveUDFType} import org.apache.hadoop.hive.ql.udf.generic._ import org.apache.hadoop.hive.ql.udf.generic.GenericUDF._ import org.apache.spark.Logging import org.apache.spark.sql.catalyst.analysis +import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.plans.logical.{Generate, Project, LogicalPlan} +import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.types._ -import org.apache.spark.sql.catalyst.analysis.MultiAlias -import org.apache.spark.sql.catalyst.errors.TreeNodeException /* Implicit conversions */ import scala.collection.JavaConversions._ @@ -189,6 +189,219 @@ private[hive] case class HiveGenericUdf(funcWrapper: HiveFunctionWrapper, childr } } +/** + * Resolves [[UnresolvedWindowFunction]] to [[HiveWindowFunction]]. + */ +private[spark] object ResolveHiveWindowFunction extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan transformUp { + case p: LogicalPlan if !p.childrenResolved => p + + // We are resolving WindowExpressions at here. When we get here, we have already + // replaced those WindowSpecReferences. + case p: LogicalPlan => + p transformExpressions { + case WindowExpression( + UnresolvedWindowFunction(name, children), + windowSpec: WindowSpecDefinition) => + // First, let's find the window function info. + val windowFunctionInfo: WindowFunctionInfo = + Option(FunctionRegistry.getWindowFunctionInfo(name.toLowerCase)).getOrElse( + throw new AnalysisException(s"Couldn't find window function $name")) + + // Get the class of this function. + // In Hive 0.12, there is no windowFunctionInfo.getFunctionClass. So, we use + // windowFunctionInfo.getfInfo().getFunctionClass for both Hive 0.13 and Hive 0.13.1. + val functionClass = windowFunctionInfo.getfInfo().getFunctionClass + val newChildren = + // Rank(), DENSE_RANK(), CUME_DIST(), and PERCENT_RANK() do not take explicit + // input parameters and requires implicit parameters, which + // are expressions in Order By clause. + if (classOf[GenericUDAFRank].isAssignableFrom(functionClass)) { + if (children.nonEmpty) { + throw new AnalysisException(s"$name does not take input parameters.") + } + windowSpec.orderSpec.map(_.child) + } else { + children + } + + // If the class is UDAF, we need to use UDAFBridge. + val isUDAFBridgeRequired = + if (classOf[UDAF].isAssignableFrom(functionClass)) { + true + } else { + false + } + + // Create the HiveWindowFunction. For the meaning of isPivotResult, see the doc of + // HiveWindowFunction. + val windowFunction = + HiveWindowFunction( + new HiveFunctionWrapper(functionClass.getName), + windowFunctionInfo.isPivotResult, + isUDAFBridgeRequired, + newChildren) + + // Second, check if the specified window function can accept window definition. + windowSpec.frameSpecification match { + case frame: SpecifiedWindowFrame if !windowFunctionInfo.isSupportsWindow => + // This Hive window function does not support user-speficied window frame. + throw new AnalysisException( + s"Window function $name does not take a frame specification.") + case frame: SpecifiedWindowFrame if windowFunctionInfo.isSupportsWindow && + windowFunctionInfo.isPivotResult => + // These two should not be true at the same time when a window frame is defined. + // If so, throw an exception. + throw new AnalysisException(s"Could not handle Hive window function $name because " + + s"it supports both a user specified window frame and pivot result.") + case _ => // OK + } + // Resolve those UnspecifiedWindowFrame because the physical Window operator still needs + // a window frame specification to work. + val newWindowSpec = windowSpec.frameSpecification match { + case UnspecifiedFrame => + val newWindowFrame = + SpecifiedWindowFrame.defaultWindowFrame( + windowSpec.orderSpec.nonEmpty, + windowFunctionInfo.isSupportsWindow) + WindowSpecDefinition(windowSpec.partitionSpec, windowSpec.orderSpec, newWindowFrame) + case _ => windowSpec + } + + // Finally, we create a WindowExpression with the resolved window function and + // specified window spec. + WindowExpression(windowFunction, newWindowSpec) + } + } +} + +/** + * A [[WindowFunction]] implementation wrapping Hive's window function. + * @param funcWrapper The wrapper for the Hive Window Function. + * @param pivotResult If it is true, the Hive function will return a list of values representing + * the values of the added columns. Otherwise, a single value is returned for + * current row. + * @param isUDAFBridgeRequired If it is true, the function returned by functionWrapper's + * createFunction is UDAF, we need to use GenericUDAFBridge to wrap + * it as a GenericUDAFResolver2. + * @param children Input parameters. + */ +private[hive] case class HiveWindowFunction( + funcWrapper: HiveFunctionWrapper, + pivotResult: Boolean, + isUDAFBridgeRequired: Boolean, + children: Seq[Expression]) extends WindowFunction + with HiveInspectors { + + // Hive window functions are based on GenericUDAFResolver2. + type UDFType = GenericUDAFResolver2 + + @transient + protected lazy val resolver: GenericUDAFResolver2 = + if (isUDAFBridgeRequired) { + new GenericUDAFBridge(funcWrapper.createFunction[UDAF]()) + } else { + funcWrapper.createFunction[GenericUDAFResolver2]() + } + + @transient + protected lazy val inputInspectors = children.map(toInspector).toArray + + // The GenericUDAFEvaluator used to evaluate the window function. + @transient + protected lazy val evaluator: GenericUDAFEvaluator = { + val parameterInfo = new SimpleGenericUDAFParameterInfo(inputInspectors, false, false) + resolver.getEvaluator(parameterInfo) + } + + // The object inspector of values returned from the Hive window function. + @transient + protected lazy val returnInspector = { + evaluator.init(GenericUDAFEvaluator.Mode.COMPLETE, inputInspectors) + } + + def dataType: DataType = + if (!pivotResult) { + inspectorToDataType(returnInspector) + } else { + // If pivotResult is true, we should take the element type out as the data type of this + // function. + inspectorToDataType(returnInspector) match { + case ArrayType(dt, _) => dt + case _ => + sys.error( + s"error resolve the data type of window function ${funcWrapper.functionClassName}") + } + } + + def nullable: Boolean = true + + override type EvaluatedType = Any + + override def eval(input: Row): Any = + throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}") + + @transient + lazy val inputProjection = new InterpretedProjection(children) + + @transient + private var hiveEvaluatorBuffer: AggregationBuffer = _ + // Output buffer. + private var outputBuffer: Any = _ + + override def init(): Unit = { + evaluator.init(GenericUDAFEvaluator.Mode.COMPLETE, inputInspectors) + } + + // Reset the hiveEvaluatorBuffer and outputPosition + override def reset(): Unit = { + // We create a new aggregation buffer to workaround the bug in GenericUDAFRowNumber. + // Basically, GenericUDAFRowNumberEvaluator.reset calls RowNumberBuffer.init. + // However, RowNumberBuffer.init does not really reset this buffer. + hiveEvaluatorBuffer = evaluator.getNewAggregationBuffer + evaluator.reset(hiveEvaluatorBuffer) + } + + override def prepareInputParameters(input: Row): AnyRef = { + wrap(inputProjection(input), inputInspectors, new Array[AnyRef](children.length)) + } + // Add input parameters for a single row. + override def update(input: AnyRef): Unit = { + evaluator.iterate(hiveEvaluatorBuffer, input.asInstanceOf[Array[AnyRef]]) + } + + override def batchUpdate(inputs: Array[AnyRef]): Unit = { + var i = 0 + while (i < inputs.length) { + evaluator.iterate(hiveEvaluatorBuffer, inputs(i).asInstanceOf[Array[AnyRef]]) + i += 1 + } + } + + override def evaluate(): Unit = { + outputBuffer = unwrap(evaluator.evaluate(hiveEvaluatorBuffer), returnInspector) + } + + override def get(index: Int): Any = { + if (!pivotResult) { + // if pivotResult is false, we will get a single value for all rows in the frame. + outputBuffer + } else { + // if pivotResult is true, we will get a Seq having the same size with the size + // of the window frame. At here, we will return the result at the position of + // index in the output buffer. + outputBuffer.asInstanceOf[Seq[Any]].get(index) + } + } + + override def toString: String = { + s"$nodeName#${funcWrapper.functionClassName}(${children.mkString(",")})" + } + + override def newInstance: WindowFunction = + new HiveWindowFunction(funcWrapper, pivotResult, isUDAFBridgeRequired, children) +} + private[hive] case class HiveGenericUdaf( funcWrapper: HiveFunctionWrapper, children: Seq[Expression]) extends AggregateExpression diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 1. testWindowing-0-327a8cd39fe30255ff492ee86f660522 b/sql/hive/src/test/resources/golden/windowing.q -- 1. testWindowing-0-327a8cd39fe30255ff492ee86f660522 new file mode 100644 index 0000000000000..850c41c8115d6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 1. testWindowing-0-327a8cd39fe30255ff492ee86f660522 @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 1 1 1173.15 +Manufacturer#1 almond antique burnished rose metallic 2 1 1 2346.3 +Manufacturer#1 almond antique chartreuse lavender yellow 34 3 2 4100.06 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 4 3 5702.650000000001 +Manufacturer#1 almond aquamarine burnished black steel 28 5 4 7117.070000000001 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 6 5 8749.730000000001 +Manufacturer#2 almond antique violet chocolate turquoise 14 1 1 1690.68 +Manufacturer#2 almond antique violet turquoise frosted 40 2 2 3491.38 +Manufacturer#2 almond aquamarine midnight light salmon 2 3 3 5523.360000000001 +Manufacturer#2 almond aquamarine rose maroon antique 25 4 4 7222.02 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5 5 8923.62 +Manufacturer#3 almond antique chartreuse khaki white 17 1 1 1671.68 +Manufacturer#3 almond antique forest lavender goldenrod 14 2 2 2861.95 +Manufacturer#3 almond antique metallic orange dim 19 3 3 4272.34 +Manufacturer#3 almond antique misty red olive 1 4 4 6195.32 +Manufacturer#3 almond antique olive coral navajo 45 5 5 7532.61 +Manufacturer#4 almond antique gainsboro frosted violet 10 1 1 1620.67 +Manufacturer#4 almond antique violet mint lemon 39 2 2 2996.09 +Manufacturer#4 almond aquamarine floral ivory bisque 27 3 3 4202.35 +Manufacturer#4 almond aquamarine yellow dodger mint 7 4 4 6047.27 +Manufacturer#4 almond azure aquamarine papaya violet 12 5 5 7337.620000000001 +Manufacturer#5 almond antique blue firebrick mint 31 1 1 1789.69 +Manufacturer#5 almond antique medium spring khaki 6 2 2 3401.3500000000004 +Manufacturer#5 almond antique sky peru orange 2 3 3 5190.08 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 4 4 6208.18 +Manufacturer#5 almond azure blanched chiffon midnight 23 5 5 7672.66 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 10. testHavingWithWindowingCondRankNoGBY-0-fef4bf638d52a9a601845347010602fd b/sql/hive/src/test/resources/golden/windowing.q -- 10. testHavingWithWindowingCondRankNoGBY-0-fef4bf638d52a9a601845347010602fd new file mode 100644 index 0000000000000..850c41c8115d6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 10. testHavingWithWindowingCondRankNoGBY-0-fef4bf638d52a9a601845347010602fd @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 1 1 1173.15 +Manufacturer#1 almond antique burnished rose metallic 2 1 1 2346.3 +Manufacturer#1 almond antique chartreuse lavender yellow 34 3 2 4100.06 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 4 3 5702.650000000001 +Manufacturer#1 almond aquamarine burnished black steel 28 5 4 7117.070000000001 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 6 5 8749.730000000001 +Manufacturer#2 almond antique violet chocolate turquoise 14 1 1 1690.68 +Manufacturer#2 almond antique violet turquoise frosted 40 2 2 3491.38 +Manufacturer#2 almond aquamarine midnight light salmon 2 3 3 5523.360000000001 +Manufacturer#2 almond aquamarine rose maroon antique 25 4 4 7222.02 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5 5 8923.62 +Manufacturer#3 almond antique chartreuse khaki white 17 1 1 1671.68 +Manufacturer#3 almond antique forest lavender goldenrod 14 2 2 2861.95 +Manufacturer#3 almond antique metallic orange dim 19 3 3 4272.34 +Manufacturer#3 almond antique misty red olive 1 4 4 6195.32 +Manufacturer#3 almond antique olive coral navajo 45 5 5 7532.61 +Manufacturer#4 almond antique gainsboro frosted violet 10 1 1 1620.67 +Manufacturer#4 almond antique violet mint lemon 39 2 2 2996.09 +Manufacturer#4 almond aquamarine floral ivory bisque 27 3 3 4202.35 +Manufacturer#4 almond aquamarine yellow dodger mint 7 4 4 6047.27 +Manufacturer#4 almond azure aquamarine papaya violet 12 5 5 7337.620000000001 +Manufacturer#5 almond antique blue firebrick mint 31 1 1 1789.69 +Manufacturer#5 almond antique medium spring khaki 6 2 2 3401.3500000000004 +Manufacturer#5 almond antique sky peru orange 2 3 3 5190.08 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 4 4 6208.18 +Manufacturer#5 almond azure blanched chiffon midnight 23 5 5 7672.66 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 11. testFirstLast-0-86bb9c97d92fdcd941bcb5143513e2e6 b/sql/hive/src/test/resources/golden/windowing.q -- 11. testFirstLast-0-86bb9c97d92fdcd941bcb5143513e2e6 new file mode 100644 index 0000000000000..921679cdcf569 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 11. testFirstLast-0-86bb9c97d92fdcd941bcb5143513e2e6 @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 2 2 34 +Manufacturer#1 almond antique burnished rose metallic 2 2 2 6 +Manufacturer#1 almond antique chartreuse lavender yellow 34 34 2 28 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 6 2 42 +Manufacturer#1 almond aquamarine burnished black steel 28 28 34 42 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 42 6 42 +Manufacturer#2 almond antique violet chocolate turquoise 14 14 14 2 +Manufacturer#2 almond antique violet turquoise frosted 40 40 14 25 +Manufacturer#2 almond aquamarine midnight light salmon 2 2 14 18 +Manufacturer#2 almond aquamarine rose maroon antique 25 25 40 18 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 18 2 18 +Manufacturer#3 almond antique chartreuse khaki white 17 17 17 19 +Manufacturer#3 almond antique forest lavender goldenrod 14 14 17 1 +Manufacturer#3 almond antique metallic orange dim 19 19 17 45 +Manufacturer#3 almond antique misty red olive 1 1 14 45 +Manufacturer#3 almond antique olive coral navajo 45 45 19 45 +Manufacturer#4 almond antique gainsboro frosted violet 10 10 10 27 +Manufacturer#4 almond antique violet mint lemon 39 39 10 7 +Manufacturer#4 almond aquamarine floral ivory bisque 27 27 10 12 +Manufacturer#4 almond aquamarine yellow dodger mint 7 7 39 12 +Manufacturer#4 almond azure aquamarine papaya violet 12 12 27 12 +Manufacturer#5 almond antique blue firebrick mint 31 31 31 2 +Manufacturer#5 almond antique medium spring khaki 6 6 31 46 +Manufacturer#5 almond antique sky peru orange 2 2 31 23 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 46 6 23 +Manufacturer#5 almond azure blanched chiffon midnight 23 23 2 23 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 12. testFirstLastWithWhere-0-84345a9f685ba63b87caa4bb16b122b5 b/sql/hive/src/test/resources/golden/windowing.q -- 12. testFirstLastWithWhere-0-84345a9f685ba63b87caa4bb16b122b5 new file mode 100644 index 0000000000000..09e30c7c57349 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 12. testFirstLastWithWhere-0-84345a9f685ba63b87caa4bb16b122b5 @@ -0,0 +1,5 @@ +Manufacturer#3 almond antique chartreuse khaki white 17 1 17 17 19 +Manufacturer#3 almond antique forest lavender goldenrod 14 2 14 17 1 +Manufacturer#3 almond antique metallic orange dim 19 3 19 17 45 +Manufacturer#3 almond antique misty red olive 1 4 1 14 45 +Manufacturer#3 almond antique olive coral navajo 45 5 45 19 45 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 13. testSumWindow-0-6cfc8840d3a4469b0fe11d63182cb59f b/sql/hive/src/test/resources/golden/windowing.q -- 13. testSumWindow-0-6cfc8840d3a4469b0fe11d63182cb59f new file mode 100644 index 0000000000000..01ee88ff23302 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 13. testSumWindow-0-6cfc8840d3a4469b0fe11d63182cb59f @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 38 2 +Manufacturer#1 almond antique burnished rose metallic 2 44 2 +Manufacturer#1 almond antique chartreuse lavender yellow 34 72 34 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 112 6 +Manufacturer#1 almond aquamarine burnished black steel 28 110 28 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 76 42 +Manufacturer#2 almond antique violet chocolate turquoise 14 56 14 +Manufacturer#2 almond antique violet turquoise frosted 40 81 40 +Manufacturer#2 almond aquamarine midnight light salmon 2 99 2 +Manufacturer#2 almond aquamarine rose maroon antique 25 85 25 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 45 18 +Manufacturer#3 almond antique chartreuse khaki white 17 50 17 +Manufacturer#3 almond antique forest lavender goldenrod 14 51 14 +Manufacturer#3 almond antique metallic orange dim 19 96 19 +Manufacturer#3 almond antique misty red olive 1 79 1 +Manufacturer#3 almond antique olive coral navajo 45 65 45 +Manufacturer#4 almond antique gainsboro frosted violet 10 76 10 +Manufacturer#4 almond antique violet mint lemon 39 83 39 +Manufacturer#4 almond aquamarine floral ivory bisque 27 95 27 +Manufacturer#4 almond aquamarine yellow dodger mint 7 85 7 +Manufacturer#4 almond azure aquamarine papaya violet 12 46 12 +Manufacturer#5 almond antique blue firebrick mint 31 39 31 +Manufacturer#5 almond antique medium spring khaki 6 85 6 +Manufacturer#5 almond antique sky peru orange 2 108 2 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 77 46 +Manufacturer#5 almond azure blanched chiffon midnight 23 71 23 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 14. testNoSortClause-0-88d96a526d3cae6ed8168c5b228974d1 b/sql/hive/src/test/resources/golden/windowing.q -- 14. testNoSortClause-0-88d96a526d3cae6ed8168c5b228974d1 new file mode 100644 index 0000000000000..c78eb640c9c27 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 14. testNoSortClause-0-88d96a526d3cae6ed8168c5b228974d1 @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 1 1 +Manufacturer#1 almond antique burnished rose metallic 2 1 1 +Manufacturer#1 almond antique chartreuse lavender yellow 34 3 2 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 4 3 +Manufacturer#1 almond aquamarine burnished black steel 28 5 4 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 6 5 +Manufacturer#2 almond antique violet chocolate turquoise 14 1 1 +Manufacturer#2 almond antique violet turquoise frosted 40 2 2 +Manufacturer#2 almond aquamarine midnight light salmon 2 3 3 +Manufacturer#2 almond aquamarine rose maroon antique 25 4 4 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5 5 +Manufacturer#3 almond antique chartreuse khaki white 17 1 1 +Manufacturer#3 almond antique forest lavender goldenrod 14 2 2 +Manufacturer#3 almond antique metallic orange dim 19 3 3 +Manufacturer#3 almond antique misty red olive 1 4 4 +Manufacturer#3 almond antique olive coral navajo 45 5 5 +Manufacturer#4 almond antique gainsboro frosted violet 10 1 1 +Manufacturer#4 almond antique violet mint lemon 39 2 2 +Manufacturer#4 almond aquamarine floral ivory bisque 27 3 3 +Manufacturer#4 almond aquamarine yellow dodger mint 7 4 4 +Manufacturer#4 almond azure aquamarine papaya violet 12 5 5 +Manufacturer#5 almond antique blue firebrick mint 31 1 1 +Manufacturer#5 almond antique medium spring khaki 6 2 2 +Manufacturer#5 almond antique sky peru orange 2 3 3 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 4 4 +Manufacturer#5 almond azure blanched chiffon midnight 23 5 5 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 15. testExpressions-0-11f6c13cf2710ce7054654cca136e73e b/sql/hive/src/test/resources/golden/windowing.q -- 15. testExpressions-0-11f6c13cf2710ce7054654cca136e73e new file mode 100644 index 0000000000000..050138ccf04ce --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 15. testExpressions-0-11f6c13cf2710ce7054654cca136e73e @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 1 1 0.3333333333333333 0.0 1 2 2.0 0.0 2 2 2 +Manufacturer#1 almond antique burnished rose metallic 2 1 1 0.3333333333333333 0.0 1 2 2.0 0.0 2 2 2 +Manufacturer#1 almond antique chartreuse lavender yellow 34 3 2 0.5 0.4 2 3 12.666666666666666 15.084944665313014 2 34 2 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 4 3 0.6666666666666666 0.6 2 4 11.0 13.379088160259652 2 6 2 +Manufacturer#1 almond aquamarine burnished black steel 28 5 4 0.8333333333333334 0.8 3 5 14.4 13.763720427268202 2 28 34 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 6 5 1.0 1.0 3 6 19.0 16.237815945091466 2 42 6 +Manufacturer#2 almond antique violet chocolate turquoise 14 1 1 0.2 0.0 1 1 14.0 0.0 4 14 14 +Manufacturer#2 almond antique violet turquoise frosted 40 2 2 0.4 0.25 1 2 27.0 13.0 4 40 14 +Manufacturer#2 almond aquamarine midnight light salmon 2 3 3 0.6 0.5 2 3 18.666666666666668 15.86050300449376 4 2 14 +Manufacturer#2 almond aquamarine rose maroon antique 25 4 4 0.8 0.75 2 4 20.25 14.00669482783144 4 25 40 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5 5 1.0 1.0 3 5 19.8 12.560254774486067 4 18 2 +Manufacturer#3 almond antique chartreuse khaki white 17 1 1 0.2 0.0 1 1 17.0 0.0 2 17 17 +Manufacturer#3 almond antique forest lavender goldenrod 14 2 2 0.4 0.25 1 2 15.5 1.5 2 14 17 +Manufacturer#3 almond antique metallic orange dim 19 3 3 0.6 0.5 2 3 16.666666666666668 2.0548046676563256 2 19 17 +Manufacturer#3 almond antique misty red olive 1 4 4 0.8 0.75 2 4 12.75 7.013380069552769 2 1 14 +Manufacturer#3 almond antique olive coral navajo 45 5 5 1.0 1.0 3 5 19.2 14.344336861632886 2 45 19 +Manufacturer#4 almond antique gainsboro frosted violet 10 1 1 0.2 0.0 1 1 10.0 0.0 0 10 10 +Manufacturer#4 almond antique violet mint lemon 39 2 2 0.4 0.25 1 2 24.5 14.5 0 39 10 +Manufacturer#4 almond aquamarine floral ivory bisque 27 3 3 0.6 0.5 2 3 25.333333333333332 11.897712198383164 0 27 10 +Manufacturer#4 almond aquamarine yellow dodger mint 7 4 4 0.8 0.75 2 4 20.75 13.007209539328564 0 7 39 +Manufacturer#4 almond azure aquamarine papaya violet 12 5 5 1.0 1.0 3 5 19.0 12.149074038789951 0 12 27 +Manufacturer#5 almond antique blue firebrick mint 31 1 1 0.2 0.0 1 1 31.0 0.0 1 31 31 +Manufacturer#5 almond antique medium spring khaki 6 2 2 0.4 0.25 1 2 18.5 12.5 1 6 31 +Manufacturer#5 almond antique sky peru orange 2 3 3 0.6 0.5 2 3 13.0 12.832251036613439 1 2 31 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 4 4 0.8 0.75 2 4 21.25 18.102140757380052 1 46 6 +Manufacturer#5 almond azure blanched chiffon midnight 23 5 5 1.0 1.0 3 5 21.6 16.206171663906314 1 23 2 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 16. testMultipleWindows-0-efd1476255eeb1b1961149144f574b7a b/sql/hive/src/test/resources/golden/windowing.q -- 16. testMultipleWindows-0-efd1476255eeb1b1961149144f574b7a new file mode 100644 index 0000000000000..c10888852b504 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 16. testMultipleWindows-0-efd1476255eeb1b1961149144f574b7a @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 1 1 0.3333333333333333 4 4 2 +Manufacturer#1 almond antique burnished rose metallic 2 1 1 0.3333333333333333 4 4 2 +Manufacturer#1 almond antique chartreuse lavender yellow 34 3 2 0.5 38 34 2 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 4 3 0.6666666666666666 44 10 2 +Manufacturer#1 almond aquamarine burnished black steel 28 5 4 0.8333333333333334 72 28 34 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 6 5 1.0 114 42 6 +Manufacturer#2 almond antique violet chocolate turquoise 14 1 1 0.2 14 14 14 +Manufacturer#2 almond antique violet turquoise frosted 40 2 2 0.4 54 40 14 +Manufacturer#2 almond aquamarine midnight light salmon 2 3 3 0.6 56 2 14 +Manufacturer#2 almond aquamarine rose maroon antique 25 4 4 0.8 81 25 40 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5 5 1.0 99 32 2 +Manufacturer#3 almond antique chartreuse khaki white 17 1 1 0.2 17 31 17 +Manufacturer#3 almond antique forest lavender goldenrod 14 2 2 0.4 31 14 17 +Manufacturer#3 almond antique metallic orange dim 19 3 3 0.6 50 50 17 +Manufacturer#3 almond antique misty red olive 1 4 4 0.8 51 1 14 +Manufacturer#3 almond antique olive coral navajo 45 5 5 1.0 96 45 19 +Manufacturer#4 almond antique gainsboro frosted violet 10 1 1 0.2 10 17 10 +Manufacturer#4 almond antique violet mint lemon 39 2 2 0.4 49 39 10 +Manufacturer#4 almond aquamarine floral ivory bisque 27 3 3 0.6 76 27 10 +Manufacturer#4 almond aquamarine yellow dodger mint 7 4 4 0.8 83 7 39 +Manufacturer#4 almond azure aquamarine papaya violet 12 5 5 1.0 95 29 27 +Manufacturer#5 almond antique blue firebrick mint 31 1 1 0.2 31 31 31 +Manufacturer#5 almond antique medium spring khaki 6 2 2 0.4 37 8 31 +Manufacturer#5 almond antique sky peru orange 2 3 3 0.6 39 2 31 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 4 4 0.8 85 46 6 +Manufacturer#5 almond azure blanched chiffon midnight 23 5 5 1.0 108 23 2 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 17. testCountStar-0-1b1fc185c8fddf68e58e92f29052ab2d b/sql/hive/src/test/resources/golden/windowing.q -- 17. testCountStar-0-1b1fc185c8fddf68e58e92f29052ab2d new file mode 100644 index 0000000000000..b1309a497d68e --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 17. testCountStar-0-1b1fc185c8fddf68e58e92f29052ab2d @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 2 2 2 +Manufacturer#1 almond antique burnished rose metallic 2 2 2 2 +Manufacturer#1 almond antique chartreuse lavender yellow 34 3 3 2 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 4 4 2 +Manufacturer#1 almond aquamarine burnished black steel 28 5 5 34 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 6 6 6 +Manufacturer#2 almond antique violet chocolate turquoise 14 1 1 14 +Manufacturer#2 almond antique violet turquoise frosted 40 2 2 14 +Manufacturer#2 almond aquamarine midnight light salmon 2 3 3 14 +Manufacturer#2 almond aquamarine rose maroon antique 25 4 4 40 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5 5 2 +Manufacturer#3 almond antique chartreuse khaki white 17 1 1 17 +Manufacturer#3 almond antique forest lavender goldenrod 14 2 2 17 +Manufacturer#3 almond antique metallic orange dim 19 3 3 17 +Manufacturer#3 almond antique misty red olive 1 4 4 14 +Manufacturer#3 almond antique olive coral navajo 45 5 5 19 +Manufacturer#4 almond antique gainsboro frosted violet 10 1 1 10 +Manufacturer#4 almond antique violet mint lemon 39 2 2 10 +Manufacturer#4 almond aquamarine floral ivory bisque 27 3 3 10 +Manufacturer#4 almond aquamarine yellow dodger mint 7 4 4 39 +Manufacturer#4 almond azure aquamarine papaya violet 12 5 5 27 +Manufacturer#5 almond antique blue firebrick mint 31 1 1 31 +Manufacturer#5 almond antique medium spring khaki 6 2 2 31 +Manufacturer#5 almond antique sky peru orange 2 3 3 31 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 4 4 6 +Manufacturer#5 almond azure blanched chiffon midnight 23 5 5 2 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 18. testUDAFs-0-6974e5959e41a661e09db18547fef58a b/sql/hive/src/test/resources/golden/windowing.q -- 18. testUDAFs-0-6974e5959e41a661e09db18547fef58a new file mode 100644 index 0000000000000..52d2ee8d0cd3f --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 18. testUDAFs-0-6974e5959e41a661e09db18547fef58a @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 4100.06 1173.15 1753.76 1366.6866666666667 +Manufacturer#1 almond antique burnished rose metallic 2 5702.650000000001 1173.15 1753.76 1425.6625000000001 +Manufacturer#1 almond antique chartreuse lavender yellow 34 7117.070000000001 1173.15 1753.76 1423.4140000000002 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 7576.58 1173.15 1753.76 1515.316 +Manufacturer#1 almond aquamarine burnished black steel 28 6403.43 1414.42 1753.76 1600.8575 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 4649.67 1414.42 1632.66 1549.89 +Manufacturer#2 almond antique violet chocolate turquoise 14 5523.360000000001 1690.68 2031.98 1841.1200000000001 +Manufacturer#2 almond antique violet turquoise frosted 40 7222.02 1690.68 2031.98 1805.505 +Manufacturer#2 almond aquamarine midnight light salmon 2 8923.62 1690.68 2031.98 1784.7240000000002 +Manufacturer#2 almond aquamarine rose maroon antique 25 7232.9400000000005 1698.66 2031.98 1808.2350000000001 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5432.24 1698.66 2031.98 1810.7466666666667 +Manufacturer#3 almond antique chartreuse khaki white 17 4272.34 1190.27 1671.68 1424.1133333333335 +Manufacturer#3 almond antique forest lavender goldenrod 14 6195.32 1190.27 1922.98 1548.83 +Manufacturer#3 almond antique metallic orange dim 19 7532.61 1190.27 1922.98 1506.522 +Manufacturer#3 almond antique misty red olive 1 5860.929999999999 1190.27 1922.98 1465.2324999999998 +Manufacturer#3 almond antique olive coral navajo 45 4670.66 1337.29 1922.98 1556.8866666666665 +Manufacturer#4 almond antique gainsboro frosted violet 10 4202.35 1206.26 1620.67 1400.7833333333335 +Manufacturer#4 almond antique violet mint lemon 39 6047.27 1206.26 1844.92 1511.8175 +Manufacturer#4 almond aquamarine floral ivory bisque 27 7337.620000000001 1206.26 1844.92 1467.5240000000001 +Manufacturer#4 almond aquamarine yellow dodger mint 7 5716.950000000001 1206.26 1844.92 1429.2375000000002 +Manufacturer#4 almond azure aquamarine papaya violet 12 4341.530000000001 1206.26 1844.92 1447.176666666667 +Manufacturer#5 almond antique blue firebrick mint 31 5190.08 1611.66 1789.69 1730.0266666666666 +Manufacturer#5 almond antique medium spring khaki 6 6208.18 1018.1 1789.69 1552.045 +Manufacturer#5 almond antique sky peru orange 2 7672.66 1018.1 1789.69 1534.532 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 5882.970000000001 1018.1 1788.73 1470.7425000000003 +Manufacturer#5 almond azure blanched chiffon midnight 23 4271.3099999999995 1018.1 1788.73 1423.7699999999998 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 19. testUDAFsWithGBY-0-67d15ee5915ac64a738fd4b60d75eb35 b/sql/hive/src/test/resources/golden/windowing.q -- 19. testUDAFsWithGBY-0-67d15ee5915ac64a738fd4b60d75eb35 new file mode 100644 index 0000000000000..6461642d34a21 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 19. testUDAFsWithGBY-0-67d15ee5915ac64a738fd4b60d75eb35 @@ -0,0 +1,25 @@ +Manufacturer#1 almond antique burnished rose metallic 2 1173.15 4529.5 1173.15 1173.15 1509.8333333333333 +Manufacturer#1 almond antique chartreuse lavender yellow 34 1753.76 5943.92 1753.76 1753.76 1485.98 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 1602.59 7576.58 1602.59 1602.59 1515.316 +Manufacturer#1 almond aquamarine burnished black steel 28 1414.42 6403.43 1414.42 1414.42 1600.8575 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 1632.66 4649.67 1632.66 1632.66 1549.89 +Manufacturer#2 almond antique violet chocolate turquoise 14 1690.68 5523.360000000001 1690.68 1690.68 1841.1200000000001 +Manufacturer#2 almond antique violet turquoise frosted 40 1800.7 7222.02 1800.7 1800.7 1805.505 +Manufacturer#2 almond aquamarine midnight light salmon 2 2031.98 8923.62 2031.98 2031.98 1784.7240000000002 +Manufacturer#2 almond aquamarine rose maroon antique 25 1698.66 7232.9400000000005 1698.66 1698.66 1808.2350000000001 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 1701.6 5432.24 1701.6 1701.6 1810.7466666666667 +Manufacturer#3 almond antique chartreuse khaki white 17 1671.68 4272.34 1671.68 1671.68 1424.1133333333335 +Manufacturer#3 almond antique forest lavender goldenrod 14 1190.27 6195.32 1190.27 1190.27 1548.83 +Manufacturer#3 almond antique metallic orange dim 19 1410.39 7532.61 1410.39 1410.39 1506.522 +Manufacturer#3 almond antique misty red olive 1 1922.98 5860.929999999999 1922.98 1922.98 1465.2324999999998 +Manufacturer#3 almond antique olive coral navajo 45 1337.29 4670.66 1337.29 1337.29 1556.8866666666665 +Manufacturer#4 almond antique gainsboro frosted violet 10 1620.67 4202.35 1620.67 1620.67 1400.7833333333335 +Manufacturer#4 almond antique violet mint lemon 39 1375.42 6047.27 1375.42 1375.42 1511.8175 +Manufacturer#4 almond aquamarine floral ivory bisque 27 1206.26 7337.620000000001 1206.26 1206.26 1467.5240000000001 +Manufacturer#4 almond aquamarine yellow dodger mint 7 1844.92 5716.950000000001 1844.92 1844.92 1429.2375000000002 +Manufacturer#4 almond azure aquamarine papaya violet 12 1290.35 4341.530000000001 1290.35 1290.35 1447.176666666667 +Manufacturer#5 almond antique blue firebrick mint 31 1789.69 5190.08 1789.69 1789.69 1730.0266666666666 +Manufacturer#5 almond antique medium spring khaki 6 1611.66 6208.18 1611.66 1611.66 1552.045 +Manufacturer#5 almond antique sky peru orange 2 1788.73 7672.66 1788.73 1788.73 1534.532 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 1018.1 5882.970000000001 1018.1 1018.1 1470.7425000000003 +Manufacturer#5 almond azure blanched chiffon midnight 23 1464.48 4271.3099999999995 1464.48 1464.48 1423.7699999999998 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 2. testGroupByWithPartitioning-0-cb5618b1e626f3a9d4a030b508b5d251 b/sql/hive/src/test/resources/golden/windowing.q -- 2. testGroupByWithPartitioning-0-cb5618b1e626f3a9d4a030b508b5d251 new file mode 100644 index 0000000000000..2c30e652aa26d --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 2. testGroupByWithPartitioning-0-cb5618b1e626f3a9d4a030b508b5d251 @@ -0,0 +1,25 @@ +Manufacturer#1 almond antique burnished rose metallic 2 1173.15 1 1 2 0 +Manufacturer#1 almond antique chartreuse lavender yellow 34 1753.76 2 2 34 32 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 1602.59 3 3 6 -28 +Manufacturer#1 almond aquamarine burnished black steel 28 1414.42 4 4 28 22 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 1632.66 5 5 42 14 +Manufacturer#2 almond antique violet chocolate turquoise 14 1690.68 1 1 14 0 +Manufacturer#2 almond antique violet turquoise frosted 40 1800.7 2 2 40 26 +Manufacturer#2 almond aquamarine midnight light salmon 2 2031.98 3 3 2 -38 +Manufacturer#2 almond aquamarine rose maroon antique 25 1698.66 4 4 25 23 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 1701.6 5 5 18 -7 +Manufacturer#3 almond antique chartreuse khaki white 17 1671.68 1 1 17 0 +Manufacturer#3 almond antique forest lavender goldenrod 14 1190.27 2 2 14 -3 +Manufacturer#3 almond antique metallic orange dim 19 1410.39 3 3 19 5 +Manufacturer#3 almond antique misty red olive 1 1922.98 4 4 1 -18 +Manufacturer#3 almond antique olive coral navajo 45 1337.29 5 5 45 44 +Manufacturer#4 almond antique gainsboro frosted violet 10 1620.67 1 1 10 0 +Manufacturer#4 almond antique violet mint lemon 39 1375.42 2 2 39 29 +Manufacturer#4 almond aquamarine floral ivory bisque 27 1206.26 3 3 27 -12 +Manufacturer#4 almond aquamarine yellow dodger mint 7 1844.92 4 4 7 -20 +Manufacturer#4 almond azure aquamarine papaya violet 12 1290.35 5 5 12 5 +Manufacturer#5 almond antique blue firebrick mint 31 1789.69 1 1 31 0 +Manufacturer#5 almond antique medium spring khaki 6 1611.66 2 2 6 -25 +Manufacturer#5 almond antique sky peru orange 2 1788.73 3 3 2 -4 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 1018.1 4 4 46 44 +Manufacturer#5 almond azure blanched chiffon midnight 23 1464.48 5 5 23 -23 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 20. testSTATs-0-da0e0cca69e42118a96b8609b8fa5838 b/sql/hive/src/test/resources/golden/windowing.q -- 20. testSTATs-0-da0e0cca69e42118a96b8609b8fa5838 new file mode 100644 index 0000000000000..1f7e8a5d67036 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 20. testSTATs-0-da0e0cca69e42118a96b8609b8fa5838 @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 273.70217881648074 273.70217881648074 [34,2] 74912.8826888888 1.0 4128.782222222221 +Manufacturer#1 almond antique burnished rose metallic 2 258.10677784349235 258.10677784349235 [34,2,6] 66619.10876874991 0.811328754177887 2801.7074999999995 +Manufacturer#1 almond antique chartreuse lavender yellow 34 230.90151585470358 230.90151585470358 [34,2,6,28] 53315.51002399992 0.695639377397664 2210.7864 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 202.73109328368946 202.73109328368946 [34,2,6,42,28] 41099.896184 0.630785977101214 2009.9536000000007 +Manufacturer#1 almond aquamarine burnished black steel 28 121.6064517973862 121.6064517973862 [34,6,42,28] 14788.129118750014 0.2036684720435979 331.1337500000004 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 96.5751586416853 96.5751586416853 [6,42,28] 9326.761266666683 -1.4442181184933883E-4 -0.20666666666708502 +Manufacturer#2 almond antique violet chocolate turquoise 14 142.2363169751898 142.2363169751898 [2,40,14] 20231.169866666663 -0.49369526554523185 -1113.7466666666658 +Manufacturer#2 almond antique violet turquoise frosted 40 137.76306498840682 137.76306498840682 [2,25,40,14] 18978.662075 -0.5205630897335946 -1004.4812499999995 +Manufacturer#2 almond aquamarine midnight light salmon 2 130.03972279269132 130.03972279269132 [2,18,25,40,14] 16910.329504000005 -0.46908967495720255 -766.1791999999995 +Manufacturer#2 almond aquamarine rose maroon antique 25 135.55100986344584 135.55100986344584 [2,18,25,40] 18374.07627499999 -0.6091405874714462 -1128.1787499999987 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 156.44019460768044 156.44019460768044 [2,18,25] 24473.534488888927 -0.9571686373491608 -1441.4466666666676 +Manufacturer#3 almond antique chartreuse khaki white 17 196.7742266885805 196.7742266885805 [17,19,14] 38720.09628888887 0.5557168646224995 224.6944444444446 +Manufacturer#3 almond antique forest lavender goldenrod 14 275.14144189852607 275.14144189852607 [17,1,19,14] 75702.81305 -0.6720833036576083 -1296.9000000000003 +Manufacturer#3 almond antique metallic orange dim 19 260.23473614412046 260.23473614412046 [17,1,19,14,45] 67722.117896 -0.5703526513979519 -2129.0664 +Manufacturer#3 almond antique misty red olive 1 275.9139962356932 275.9139962356932 [1,19,14,45] 76128.53331875012 -0.577476899644802 -2547.7868749999993 +Manufacturer#3 almond antique olive coral navajo 45 260.5815918713796 260.5815918713796 [1,19,45] 67902.76602222225 -0.8710736366736884 -4099.731111111111 +Manufacturer#4 almond antique gainsboro frosted violet 10 170.13011889596618 170.13011889596618 [39,27,10] 28944.25735555559 -0.6656975320098423 -1347.4777777777779 +Manufacturer#4 almond antique violet mint lemon 39 242.26834609323197 242.26834609323197 [39,7,27,10] 58693.95151875002 -0.8051852719193339 -2537.328125 +Manufacturer#4 almond aquamarine floral ivory bisque 27 234.10001662537326 234.10001662537326 [39,7,27,10,12] 54802.817784000035 -0.6046935574240581 -1719.8079999999995 +Manufacturer#4 almond aquamarine yellow dodger mint 7 247.3342714197732 247.3342714197732 [39,7,27,12] 61174.24181875003 -0.5508665654707869 -1719.0368749999975 +Manufacturer#4 almond azure aquamarine papaya violet 12 283.3344330566893 283.3344330566893 [7,27,12] 80278.40095555557 -0.7755740084632333 -1867.4888888888881 +Manufacturer#5 almond antique blue firebrick mint 31 83.69879024746363 83.69879024746363 [2,6,31] 7005.487488888913 0.39004303087285047 418.9233333333353 +Manufacturer#5 almond antique medium spring khaki 6 316.68049612345885 316.68049612345885 [2,6,46,31] 100286.53662500004 -0.713612911776183 -4090.853749999999 +Manufacturer#5 almond antique sky peru orange 2 285.40506298242155 285.40506298242155 [2,23,6,46,31] 81456.04997600002 -0.712858514567818 -3297.2011999999986 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 285.43749038756283 285.43749038756283 [2,23,6,46] 81474.56091875004 -0.984128787153391 -4871.028125000002 +Manufacturer#5 almond azure blanched chiffon midnight 23 315.9225931564038 315.9225931564038 [2,23,46] 99807.08486666664 -0.9978877469246936 -5664.856666666666 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 21. testDISTs-0-672d4cb385b7ced2e446f132474293ad b/sql/hive/src/test/resources/golden/windowing.q -- 21. testDISTs-0-672d4cb385b7ced2e446f132474293ad new file mode 100644 index 0000000000000..e7c39f454fb37 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 21. testDISTs-0-672d4cb385b7ced2e446f132474293ad @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 [{"x":1173.15,"y":2.0},{"x":1753.76,"y":1.0}] 121152.0 1 +Manufacturer#1 almond antique burnished rose metallic 2 [{"x":1173.15,"y":2.0},{"x":1602.59,"y":1.0},{"x":1753.76,"y":1.0}] 115872.0 2 +Manufacturer#1 almond antique chartreuse lavender yellow 34 [{"x":1173.15,"y":2.0},{"x":1414.42,"y":1.0},{"x":1602.59,"y":1.0},{"x":1753.76,"y":1.0}] 110592.0 3 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 [{"x":1173.15,"y":1.0},{"x":1414.42,"y":1.0},{"x":1602.59,"y":1.0},{"x":1632.66,"y":1.0},{"x":1753.76,"y":1.0}] 86428.0 4 +Manufacturer#1 almond aquamarine burnished black steel 28 [{"x":1414.42,"y":1.0},{"x":1602.59,"y":1.0},{"x":1632.66,"y":1.0},{"x":1753.76,"y":1.0}] 86098.0 5 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 [{"x":1414.42,"y":1.0},{"x":1602.59,"y":1.0},{"x":1632.66,"y":1.0}] 86428.0 6 +Manufacturer#2 almond antique violet chocolate turquoise 14 [{"x":1690.68,"y":1.0},{"x":1800.7,"y":1.0},{"x":2031.98,"y":1.0}] 146985.0 1 +Manufacturer#2 almond antique violet turquoise frosted 40 [{"x":1690.68,"y":1.0},{"x":1698.66,"y":1.0},{"x":1800.7,"y":1.0},{"x":2031.98,"y":1.0}] 139825.5 2 +Manufacturer#2 almond aquamarine midnight light salmon 2 [{"x":1690.68,"y":1.0},{"x":1698.66,"y":1.0},{"x":1701.6,"y":1.0},{"x":1800.7,"y":1.0},{"x":2031.98,"y":1.0}] 146985.0 3 +Manufacturer#2 almond aquamarine rose maroon antique 25 [{"x":1698.66,"y":1.0},{"x":1701.6,"y":1.0},{"x":1800.7,"y":1.0},{"x":2031.98,"y":1.0}] 169347.0 4 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 [{"x":1698.66,"y":1.0},{"x":1701.6,"y":1.0},{"x":2031.98,"y":1.0}] 146985.0 5 +Manufacturer#3 almond antique chartreuse khaki white 17 [{"x":1190.27,"y":1.0},{"x":1410.39,"y":1.0},{"x":1671.68,"y":1.0}] 90681.0 1 +Manufacturer#3 almond antique forest lavender goldenrod 14 [{"x":1190.27,"y":1.0},{"x":1410.39,"y":1.0},{"x":1671.68,"y":1.0},{"x":1922.98,"y":1.0}] 65831.5 2 +Manufacturer#3 almond antique metallic orange dim 19 [{"x":1190.27,"y":1.0},{"x":1337.29,"y":1.0},{"x":1410.39,"y":1.0},{"x":1671.68,"y":1.0},{"x":1922.98,"y":1.0}] 90681.0 3 +Manufacturer#3 almond antique misty red olive 1 [{"x":1190.27,"y":1.0},{"x":1337.29,"y":1.0},{"x":1410.39,"y":1.0},{"x":1922.98,"y":1.0}] 76690.0 4 +Manufacturer#3 almond antique olive coral navajo 45 [{"x":1337.29,"y":1.0},{"x":1410.39,"y":1.0},{"x":1922.98,"y":1.0}] 112398.0 5 +Manufacturer#4 almond antique gainsboro frosted violet 10 [{"x":1206.26,"y":1.0},{"x":1375.42,"y":1.0},{"x":1620.67,"y":1.0}] 48427.0 1 +Manufacturer#4 almond antique violet mint lemon 39 [{"x":1206.26,"y":1.0},{"x":1375.42,"y":1.0},{"x":1620.67,"y":1.0},{"x":1844.92,"y":1.0}] 46844.0 2 +Manufacturer#4 almond aquamarine floral ivory bisque 27 [{"x":1206.26,"y":1.0},{"x":1290.35,"y":1.0},{"x":1375.42,"y":1.0},{"x":1620.67,"y":1.0},{"x":1844.92,"y":1.0}] 45261.0 3 +Manufacturer#4 almond aquamarine yellow dodger mint 7 [{"x":1206.26,"y":1.0},{"x":1290.35,"y":1.0},{"x":1375.42,"y":1.0},{"x":1844.92,"y":1.0}] 39309.0 4 +Manufacturer#4 almond azure aquamarine papaya violet 12 [{"x":1206.26,"y":1.0},{"x":1290.35,"y":1.0},{"x":1844.92,"y":1.0}] 33357.0 5 +Manufacturer#5 almond antique blue firebrick mint 31 [{"x":1611.66,"y":1.0},{"x":1788.73,"y":1.0},{"x":1789.69,"y":1.0}] 155733.0 1 +Manufacturer#5 almond antique medium spring khaki 6 [{"x":1018.1,"y":1.0},{"x":1611.66,"y":1.0},{"x":1788.73,"y":1.0},{"x":1789.69,"y":1.0}] 99201.0 2 +Manufacturer#5 almond antique sky peru orange 2 [{"x":1018.1,"y":1.0},{"x":1464.48,"y":1.0},{"x":1611.66,"y":1.0},{"x":1788.73,"y":1.0},{"x":1789.69,"y":1.0}] 78486.0 3 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 [{"x":1018.1,"y":1.0},{"x":1464.48,"y":1.0},{"x":1611.66,"y":1.0},{"x":1788.73,"y":1.0}] 60577.5 4 +Manufacturer#5 almond azure blanched chiffon midnight 23 [{"x":1018.1,"y":1.0},{"x":1464.48,"y":1.0},{"x":1788.73,"y":1.0}] 78486.0 5 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 24. testLateralViews-0-dea06072f0a64fe4537fae854944ed5a b/sql/hive/src/test/resources/golden/windowing.q -- 24. testLateralViews-0-dea06072f0a64fe4537fae854944ed5a new file mode 100644 index 0000000000000..dc83c9fffe932 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 24. testLateralViews-0-dea06072f0a64fe4537fae854944ed5a @@ -0,0 +1,78 @@ +Manufacturer#1 almond antique burnished rose metallic 1 2 2 +Manufacturer#1 almond antique burnished rose metallic 1 2 4 +Manufacturer#1 almond antique burnished rose metallic 2 2 6 +Manufacturer#1 almond antique burnished rose metallic 2 2 6 +Manufacturer#1 almond antique burnished rose metallic 3 2 6 +Manufacturer#1 almond antique burnished rose metallic 3 2 6 +Manufacturer#1 almond antique salmon chartreuse burlywood 1 6 10 +Manufacturer#1 almond antique salmon chartreuse burlywood 2 6 14 +Manufacturer#1 almond antique salmon chartreuse burlywood 3 6 18 +Manufacturer#1 almond aquamarine burnished black steel 1 28 40 +Manufacturer#1 almond aquamarine burnished black steel 2 28 62 +Manufacturer#1 almond aquamarine burnished black steel 3 28 84 +Manufacturer#1 almond antique chartreuse lavender yellow 1 34 90 +Manufacturer#1 almond antique chartreuse lavender yellow 2 34 96 +Manufacturer#1 almond antique chartreuse lavender yellow 3 34 102 +Manufacturer#1 almond aquamarine pink moccasin thistle 1 42 110 +Manufacturer#1 almond aquamarine pink moccasin thistle 2 42 118 +Manufacturer#1 almond aquamarine pink moccasin thistle 3 42 126 +Manufacturer#2 almond aquamarine midnight light salmon 1 2 2 +Manufacturer#2 almond aquamarine midnight light salmon 2 2 4 +Manufacturer#2 almond aquamarine midnight light salmon 3 2 6 +Manufacturer#2 almond antique violet chocolate turquoise 1 14 18 +Manufacturer#2 almond antique violet chocolate turquoise 2 14 30 +Manufacturer#2 almond antique violet chocolate turquoise 3 14 42 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 1 18 46 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 2 18 50 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 3 18 54 +Manufacturer#2 almond aquamarine rose maroon antique 1 25 61 +Manufacturer#2 almond aquamarine rose maroon antique 2 25 68 +Manufacturer#2 almond aquamarine rose maroon antique 3 25 75 +Manufacturer#2 almond antique violet turquoise frosted 1 40 90 +Manufacturer#2 almond antique violet turquoise frosted 2 40 105 +Manufacturer#2 almond antique violet turquoise frosted 3 40 120 +Manufacturer#3 almond antique misty red olive 1 1 1 +Manufacturer#3 almond antique misty red olive 2 1 2 +Manufacturer#3 almond antique misty red olive 3 1 3 +Manufacturer#3 almond antique forest lavender goldenrod 1 14 16 +Manufacturer#3 almond antique forest lavender goldenrod 2 14 29 +Manufacturer#3 almond antique forest lavender goldenrod 3 14 42 +Manufacturer#3 almond antique chartreuse khaki white 1 17 45 +Manufacturer#3 almond antique chartreuse khaki white 2 17 48 +Manufacturer#3 almond antique chartreuse khaki white 3 17 51 +Manufacturer#3 almond antique metallic orange dim 1 19 53 +Manufacturer#3 almond antique metallic orange dim 2 19 55 +Manufacturer#3 almond antique metallic orange dim 3 19 57 +Manufacturer#3 almond antique olive coral navajo 1 45 83 +Manufacturer#3 almond antique olive coral navajo 2 45 109 +Manufacturer#3 almond antique olive coral navajo 3 45 135 +Manufacturer#4 almond aquamarine yellow dodger mint 1 7 7 +Manufacturer#4 almond aquamarine yellow dodger mint 2 7 14 +Manufacturer#4 almond aquamarine yellow dodger mint 3 7 21 +Manufacturer#4 almond antique gainsboro frosted violet 1 10 24 +Manufacturer#4 almond antique gainsboro frosted violet 2 10 27 +Manufacturer#4 almond antique gainsboro frosted violet 3 10 30 +Manufacturer#4 almond azure aquamarine papaya violet 1 12 32 +Manufacturer#4 almond azure aquamarine papaya violet 2 12 34 +Manufacturer#4 almond azure aquamarine papaya violet 3 12 36 +Manufacturer#4 almond aquamarine floral ivory bisque 1 27 51 +Manufacturer#4 almond aquamarine floral ivory bisque 2 27 66 +Manufacturer#4 almond aquamarine floral ivory bisque 3 27 81 +Manufacturer#4 almond antique violet mint lemon 1 39 93 +Manufacturer#4 almond antique violet mint lemon 2 39 105 +Manufacturer#4 almond antique violet mint lemon 3 39 117 +Manufacturer#5 almond antique sky peru orange 1 2 2 +Manufacturer#5 almond antique sky peru orange 2 2 4 +Manufacturer#5 almond antique sky peru orange 3 2 6 +Manufacturer#5 almond antique medium spring khaki 1 6 10 +Manufacturer#5 almond antique medium spring khaki 2 6 14 +Manufacturer#5 almond antique medium spring khaki 3 6 18 +Manufacturer#5 almond azure blanched chiffon midnight 1 23 35 +Manufacturer#5 almond azure blanched chiffon midnight 2 23 52 +Manufacturer#5 almond azure blanched chiffon midnight 3 23 69 +Manufacturer#5 almond antique blue firebrick mint 1 31 77 +Manufacturer#5 almond antique blue firebrick mint 2 31 85 +Manufacturer#5 almond antique blue firebrick mint 3 31 93 +Manufacturer#5 almond aquamarine dodger light gainsboro 1 46 108 +Manufacturer#5 almond aquamarine dodger light gainsboro 2 46 123 +Manufacturer#5 almond aquamarine dodger light gainsboro 3 46 138 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 26. testGroupByHavingWithSWQAndAlias-0-b996a664b06e5741c08079d5c38241bc b/sql/hive/src/test/resources/golden/windowing.q -- 26. testGroupByHavingWithSWQAndAlias-0-b996a664b06e5741c08079d5c38241bc new file mode 100644 index 0000000000000..2c30e652aa26d --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 26. testGroupByHavingWithSWQAndAlias-0-b996a664b06e5741c08079d5c38241bc @@ -0,0 +1,25 @@ +Manufacturer#1 almond antique burnished rose metallic 2 1173.15 1 1 2 0 +Manufacturer#1 almond antique chartreuse lavender yellow 34 1753.76 2 2 34 32 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 1602.59 3 3 6 -28 +Manufacturer#1 almond aquamarine burnished black steel 28 1414.42 4 4 28 22 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 1632.66 5 5 42 14 +Manufacturer#2 almond antique violet chocolate turquoise 14 1690.68 1 1 14 0 +Manufacturer#2 almond antique violet turquoise frosted 40 1800.7 2 2 40 26 +Manufacturer#2 almond aquamarine midnight light salmon 2 2031.98 3 3 2 -38 +Manufacturer#2 almond aquamarine rose maroon antique 25 1698.66 4 4 25 23 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 1701.6 5 5 18 -7 +Manufacturer#3 almond antique chartreuse khaki white 17 1671.68 1 1 17 0 +Manufacturer#3 almond antique forest lavender goldenrod 14 1190.27 2 2 14 -3 +Manufacturer#3 almond antique metallic orange dim 19 1410.39 3 3 19 5 +Manufacturer#3 almond antique misty red olive 1 1922.98 4 4 1 -18 +Manufacturer#3 almond antique olive coral navajo 45 1337.29 5 5 45 44 +Manufacturer#4 almond antique gainsboro frosted violet 10 1620.67 1 1 10 0 +Manufacturer#4 almond antique violet mint lemon 39 1375.42 2 2 39 29 +Manufacturer#4 almond aquamarine floral ivory bisque 27 1206.26 3 3 27 -12 +Manufacturer#4 almond aquamarine yellow dodger mint 7 1844.92 4 4 7 -20 +Manufacturer#4 almond azure aquamarine papaya violet 12 1290.35 5 5 12 5 +Manufacturer#5 almond antique blue firebrick mint 31 1789.69 1 1 31 0 +Manufacturer#5 almond antique medium spring khaki 6 1611.66 2 2 6 -25 +Manufacturer#5 almond antique sky peru orange 2 1788.73 3 3 2 -4 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 1018.1 4 4 46 44 +Manufacturer#5 almond azure blanched chiffon midnight 23 1464.48 5 5 23 -23 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 27. testMultipleRangeWindows-0-227e080e337d734dd88ff814b3b412e4 b/sql/hive/src/test/resources/golden/windowing.q -- 27. testMultipleRangeWindows-0-227e080e337d734dd88ff814b3b412e4 new file mode 100644 index 0000000000000..b2a91ba727a72 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 27. testMultipleRangeWindows-0-227e080e337d734dd88ff814b3b412e4 @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 4 10 +Manufacturer#1 almond antique burnished rose metallic 2 4 10 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 10 6 +Manufacturer#1 almond aquamarine burnished black steel 28 28 62 +Manufacturer#1 almond antique chartreuse lavender yellow 34 62 76 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 76 42 +Manufacturer#2 almond aquamarine midnight light salmon 2 2 2 +Manufacturer#2 almond antique violet chocolate turquoise 14 14 32 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 32 43 +Manufacturer#2 almond aquamarine rose maroon antique 25 43 25 +Manufacturer#2 almond antique violet turquoise frosted 40 40 40 +Manufacturer#3 almond antique misty red olive 1 1 1 +Manufacturer#3 almond antique forest lavender goldenrod 14 14 50 +Manufacturer#3 almond antique chartreuse khaki white 17 31 36 +Manufacturer#3 almond antique metallic orange dim 19 50 19 +Manufacturer#3 almond antique olive coral navajo 45 45 45 +Manufacturer#4 almond aquamarine yellow dodger mint 7 7 29 +Manufacturer#4 almond antique gainsboro frosted violet 10 17 22 +Manufacturer#4 almond azure aquamarine papaya violet 12 29 12 +Manufacturer#4 almond aquamarine floral ivory bisque 27 27 27 +Manufacturer#4 almond antique violet mint lemon 39 39 39 +Manufacturer#5 almond antique sky peru orange 2 2 8 +Manufacturer#5 almond antique medium spring khaki 6 8 6 +Manufacturer#5 almond azure blanched chiffon midnight 23 23 54 +Manufacturer#5 almond antique blue firebrick mint 31 54 31 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 46 46 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 28. testPartOrderInUDAFInvoke-0-25912ae7d18c91cc09e17e57968fb5db b/sql/hive/src/test/resources/golden/windowing.q -- 28. testPartOrderInUDAFInvoke-0-25912ae7d18c91cc09e17e57968fb5db new file mode 100644 index 0000000000000..5bcb0fa941d65 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 28. testPartOrderInUDAFInvoke-0-25912ae7d18c91cc09e17e57968fb5db @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 38 +Manufacturer#1 almond antique burnished rose metallic 2 44 +Manufacturer#1 almond antique chartreuse lavender yellow 34 72 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 112 +Manufacturer#1 almond aquamarine burnished black steel 28 110 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 76 +Manufacturer#2 almond antique violet chocolate turquoise 14 56 +Manufacturer#2 almond antique violet turquoise frosted 40 81 +Manufacturer#2 almond aquamarine midnight light salmon 2 99 +Manufacturer#2 almond aquamarine rose maroon antique 25 85 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 45 +Manufacturer#3 almond antique chartreuse khaki white 17 50 +Manufacturer#3 almond antique forest lavender goldenrod 14 51 +Manufacturer#3 almond antique metallic orange dim 19 96 +Manufacturer#3 almond antique misty red olive 1 79 +Manufacturer#3 almond antique olive coral navajo 45 65 +Manufacturer#4 almond antique gainsboro frosted violet 10 76 +Manufacturer#4 almond antique violet mint lemon 39 83 +Manufacturer#4 almond aquamarine floral ivory bisque 27 95 +Manufacturer#4 almond aquamarine yellow dodger mint 7 85 +Manufacturer#4 almond azure aquamarine papaya violet 12 46 +Manufacturer#5 almond antique blue firebrick mint 31 39 +Manufacturer#5 almond antique medium spring khaki 6 85 +Manufacturer#5 almond antique sky peru orange 2 108 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 77 +Manufacturer#5 almond azure blanched chiffon midnight 23 71 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 29. testPartOrderInWdwDef-0-88945892370ccbc1125a927a3d55342a b/sql/hive/src/test/resources/golden/windowing.q -- 29. testPartOrderInWdwDef-0-88945892370ccbc1125a927a3d55342a new file mode 100644 index 0000000000000..5bcb0fa941d65 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 29. testPartOrderInWdwDef-0-88945892370ccbc1125a927a3d55342a @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 38 +Manufacturer#1 almond antique burnished rose metallic 2 44 +Manufacturer#1 almond antique chartreuse lavender yellow 34 72 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 112 +Manufacturer#1 almond aquamarine burnished black steel 28 110 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 76 +Manufacturer#2 almond antique violet chocolate turquoise 14 56 +Manufacturer#2 almond antique violet turquoise frosted 40 81 +Manufacturer#2 almond aquamarine midnight light salmon 2 99 +Manufacturer#2 almond aquamarine rose maroon antique 25 85 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 45 +Manufacturer#3 almond antique chartreuse khaki white 17 50 +Manufacturer#3 almond antique forest lavender goldenrod 14 51 +Manufacturer#3 almond antique metallic orange dim 19 96 +Manufacturer#3 almond antique misty red olive 1 79 +Manufacturer#3 almond antique olive coral navajo 45 65 +Manufacturer#4 almond antique gainsboro frosted violet 10 76 +Manufacturer#4 almond antique violet mint lemon 39 83 +Manufacturer#4 almond aquamarine floral ivory bisque 27 95 +Manufacturer#4 almond aquamarine yellow dodger mint 7 85 +Manufacturer#4 almond azure aquamarine papaya violet 12 46 +Manufacturer#5 almond antique blue firebrick mint 31 39 +Manufacturer#5 almond antique medium spring khaki 6 85 +Manufacturer#5 almond antique sky peru orange 2 108 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 77 +Manufacturer#5 almond azure blanched chiffon midnight 23 71 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 3. testGroupByHavingWithSWQ-0-a5a5339330a6a6660d32ccb0cc5d7100 b/sql/hive/src/test/resources/golden/windowing.q -- 3. testGroupByHavingWithSWQ-0-a5a5339330a6a6660d32ccb0cc5d7100 new file mode 100644 index 0000000000000..2c30e652aa26d --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 3. testGroupByHavingWithSWQ-0-a5a5339330a6a6660d32ccb0cc5d7100 @@ -0,0 +1,25 @@ +Manufacturer#1 almond antique burnished rose metallic 2 1173.15 1 1 2 0 +Manufacturer#1 almond antique chartreuse lavender yellow 34 1753.76 2 2 34 32 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 1602.59 3 3 6 -28 +Manufacturer#1 almond aquamarine burnished black steel 28 1414.42 4 4 28 22 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 1632.66 5 5 42 14 +Manufacturer#2 almond antique violet chocolate turquoise 14 1690.68 1 1 14 0 +Manufacturer#2 almond antique violet turquoise frosted 40 1800.7 2 2 40 26 +Manufacturer#2 almond aquamarine midnight light salmon 2 2031.98 3 3 2 -38 +Manufacturer#2 almond aquamarine rose maroon antique 25 1698.66 4 4 25 23 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 1701.6 5 5 18 -7 +Manufacturer#3 almond antique chartreuse khaki white 17 1671.68 1 1 17 0 +Manufacturer#3 almond antique forest lavender goldenrod 14 1190.27 2 2 14 -3 +Manufacturer#3 almond antique metallic orange dim 19 1410.39 3 3 19 5 +Manufacturer#3 almond antique misty red olive 1 1922.98 4 4 1 -18 +Manufacturer#3 almond antique olive coral navajo 45 1337.29 5 5 45 44 +Manufacturer#4 almond antique gainsboro frosted violet 10 1620.67 1 1 10 0 +Manufacturer#4 almond antique violet mint lemon 39 1375.42 2 2 39 29 +Manufacturer#4 almond aquamarine floral ivory bisque 27 1206.26 3 3 27 -12 +Manufacturer#4 almond aquamarine yellow dodger mint 7 1844.92 4 4 7 -20 +Manufacturer#4 almond azure aquamarine papaya violet 12 1290.35 5 5 12 5 +Manufacturer#5 almond antique blue firebrick mint 31 1789.69 1 1 31 0 +Manufacturer#5 almond antique medium spring khaki 6 1611.66 2 2 6 -25 +Manufacturer#5 almond antique sky peru orange 2 1788.73 3 3 2 -4 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 1018.1 4 4 46 44 +Manufacturer#5 almond azure blanched chiffon midnight 23 1464.48 5 5 23 -23 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 30. testDefaultPartitioningSpecRules-0-fa80b09c99e3c1487de48ea71a88dada b/sql/hive/src/test/resources/golden/windowing.q -- 30. testDefaultPartitioningSpecRules-0-fa80b09c99e3c1487de48ea71a88dada new file mode 100644 index 0000000000000..698a44349d2a6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 30. testDefaultPartitioningSpecRules-0-fa80b09c99e3c1487de48ea71a88dada @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 38 4 +Manufacturer#1 almond antique burnished rose metallic 2 44 4 +Manufacturer#1 almond antique chartreuse lavender yellow 34 72 38 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 112 44 +Manufacturer#1 almond aquamarine burnished black steel 28 110 72 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 76 114 +Manufacturer#2 almond antique violet chocolate turquoise 14 56 14 +Manufacturer#2 almond antique violet turquoise frosted 40 81 54 +Manufacturer#2 almond aquamarine midnight light salmon 2 99 56 +Manufacturer#2 almond aquamarine rose maroon antique 25 85 81 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 45 99 +Manufacturer#3 almond antique chartreuse khaki white 17 50 17 +Manufacturer#3 almond antique forest lavender goldenrod 14 51 31 +Manufacturer#3 almond antique metallic orange dim 19 96 50 +Manufacturer#3 almond antique misty red olive 1 79 51 +Manufacturer#3 almond antique olive coral navajo 45 65 96 +Manufacturer#4 almond antique gainsboro frosted violet 10 76 10 +Manufacturer#4 almond antique violet mint lemon 39 83 49 +Manufacturer#4 almond aquamarine floral ivory bisque 27 95 76 +Manufacturer#4 almond aquamarine yellow dodger mint 7 85 83 +Manufacturer#4 almond azure aquamarine papaya violet 12 46 95 +Manufacturer#5 almond antique blue firebrick mint 31 39 31 +Manufacturer#5 almond antique medium spring khaki 6 85 37 +Manufacturer#5 almond antique sky peru orange 2 108 39 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 77 85 +Manufacturer#5 almond azure blanched chiffon midnight 23 71 108 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 36. testRankWithPartitioning-0-45ccbaf0ee083858f7661c66b11d4768 b/sql/hive/src/test/resources/golden/windowing.q -- 36. testRankWithPartitioning-0-45ccbaf0ee083858f7661c66b11d4768 new file mode 100644 index 0000000000000..e35257d98382e --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 36. testRankWithPartitioning-0-45ccbaf0ee083858f7661c66b11d4768 @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 1 +Manufacturer#1 almond antique burnished rose metallic 2 1 +Manufacturer#1 almond antique chartreuse lavender yellow 34 3 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 4 +Manufacturer#1 almond aquamarine burnished black steel 28 5 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 6 +Manufacturer#2 almond antique violet chocolate turquoise 14 1 +Manufacturer#2 almond antique violet turquoise frosted 40 2 +Manufacturer#2 almond aquamarine midnight light salmon 2 3 +Manufacturer#2 almond aquamarine rose maroon antique 25 4 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5 +Manufacturer#3 almond antique chartreuse khaki white 17 1 +Manufacturer#3 almond antique forest lavender goldenrod 14 2 +Manufacturer#3 almond antique metallic orange dim 19 3 +Manufacturer#3 almond antique misty red olive 1 4 +Manufacturer#3 almond antique olive coral navajo 45 5 +Manufacturer#4 almond antique gainsboro frosted violet 10 1 +Manufacturer#4 almond antique violet mint lemon 39 2 +Manufacturer#4 almond aquamarine floral ivory bisque 27 3 +Manufacturer#4 almond aquamarine yellow dodger mint 7 4 +Manufacturer#4 almond azure aquamarine papaya violet 12 5 +Manufacturer#5 almond antique blue firebrick mint 31 1 +Manufacturer#5 almond antique medium spring khaki 6 2 +Manufacturer#5 almond antique sky peru orange 2 3 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 4 +Manufacturer#5 almond azure blanched chiffon midnight 23 5 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 37. testPartitioningVariousForms-0-3436e50214f9afdec84334e10faa931a b/sql/hive/src/test/resources/golden/windowing.q -- 37. testPartitioningVariousForms-0-3436e50214f9afdec84334e10faa931a new file mode 100644 index 0000000000000..9c0ca6c7a00ba --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 37. testPartitioningVariousForms-0-3436e50214f9afdec84334e10faa931a @@ -0,0 +1,26 @@ +Manufacturer#1 8749.73 1173.15 1753.76 1458.29 6 +Manufacturer#1 8749.73 1173.15 1753.76 1458.29 6 +Manufacturer#1 8749.73 1173.15 1753.76 1458.29 6 +Manufacturer#1 8749.73 1173.15 1753.76 1458.29 6 +Manufacturer#1 8749.73 1173.15 1753.76 1458.29 6 +Manufacturer#1 8749.73 1173.15 1753.76 1458.29 6 +Manufacturer#2 8923.62 1690.68 2031.98 1784.72 5 +Manufacturer#2 8923.62 1690.68 2031.98 1784.72 5 +Manufacturer#2 8923.62 1690.68 2031.98 1784.72 5 +Manufacturer#2 8923.62 1690.68 2031.98 1784.72 5 +Manufacturer#2 8923.62 1690.68 2031.98 1784.72 5 +Manufacturer#3 7532.61 1190.27 1922.98 1506.52 5 +Manufacturer#3 7532.61 1190.27 1922.98 1506.52 5 +Manufacturer#3 7532.61 1190.27 1922.98 1506.52 5 +Manufacturer#3 7532.61 1190.27 1922.98 1506.52 5 +Manufacturer#3 7532.61 1190.27 1922.98 1506.52 5 +Manufacturer#4 7337.62 1206.26 1844.92 1467.52 5 +Manufacturer#4 7337.62 1206.26 1844.92 1467.52 5 +Manufacturer#4 7337.62 1206.26 1844.92 1467.52 5 +Manufacturer#4 7337.62 1206.26 1844.92 1467.52 5 +Manufacturer#4 7337.62 1206.26 1844.92 1467.52 5 +Manufacturer#5 7672.66 1018.1 1789.69 1534.53 5 +Manufacturer#5 7672.66 1018.1 1789.69 1534.53 5 +Manufacturer#5 7672.66 1018.1 1789.69 1534.53 5 +Manufacturer#5 7672.66 1018.1 1789.69 1534.53 5 +Manufacturer#5 7672.66 1018.1 1789.69 1534.53 5 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 38. testPartitioningVariousForms2-0-cba9d84a6b1bb5e36595338d4602377e b/sql/hive/src/test/resources/golden/windowing.q -- 38. testPartitioningVariousForms2-0-cba9d84a6b1bb5e36595338d4602377e new file mode 100644 index 0000000000000..fc27df2f2b648 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 38. testPartitioningVariousForms2-0-cba9d84a6b1bb5e36595338d4602377e @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 1173.15 1173.15 1173.15 +Manufacturer#1 almond antique burnished rose metallic 2 2346.3 1173.15 1173.15 +Manufacturer#1 almond antique chartreuse lavender yellow 34 1753.76 1753.76 1753.76 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 1602.59 1602.59 1602.59 +Manufacturer#1 almond aquamarine burnished black steel 28 1414.42 1414.42 1414.42 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 1632.66 1632.66 1632.66 +Manufacturer#2 almond antique violet chocolate turquoise 14 1690.68 1690.68 1690.68 +Manufacturer#2 almond antique violet turquoise frosted 40 1800.7 1800.7 1800.7 +Manufacturer#2 almond aquamarine midnight light salmon 2 2031.98 2031.98 2031.98 +Manufacturer#2 almond aquamarine rose maroon antique 25 1698.66 1698.66 1698.66 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 1701.6 1701.6 1701.6 +Manufacturer#3 almond antique chartreuse khaki white 17 1671.68 1671.68 1671.68 +Manufacturer#3 almond antique forest lavender goldenrod 14 1190.27 1190.27 1190.27 +Manufacturer#3 almond antique metallic orange dim 19 1410.39 1410.39 1410.39 +Manufacturer#3 almond antique misty red olive 1 1922.98 1922.98 1922.98 +Manufacturer#3 almond antique olive coral navajo 45 1337.29 1337.29 1337.29 +Manufacturer#4 almond antique gainsboro frosted violet 10 1620.67 1620.67 1620.67 +Manufacturer#4 almond antique violet mint lemon 39 1375.42 1375.42 1375.42 +Manufacturer#4 almond aquamarine floral ivory bisque 27 1206.26 1206.26 1206.26 +Manufacturer#4 almond aquamarine yellow dodger mint 7 1844.92 1844.92 1844.92 +Manufacturer#4 almond azure aquamarine papaya violet 12 1290.35 1290.35 1290.35 +Manufacturer#5 almond antique blue firebrick mint 31 1789.69 1789.69 1789.69 +Manufacturer#5 almond antique medium spring khaki 6 1611.66 1611.66 1611.66 +Manufacturer#5 almond antique sky peru orange 2 1788.73 1788.73 1788.73 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 1018.1 1018.1 1018.1 +Manufacturer#5 almond azure blanched chiffon midnight 23 1464.48 1464.48 1464.48 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 39. testUDFOnOrderCols-0-7647562850dd367ef1e6c63117805423 b/sql/hive/src/test/resources/golden/windowing.q -- 39. testUDFOnOrderCols-0-7647562850dd367ef1e6c63117805423 new file mode 100644 index 0000000000000..e5a541f56f6f3 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 39. testUDFOnOrderCols-0-7647562850dd367ef1e6c63117805423 @@ -0,0 +1,26 @@ +Manufacturer#1 LARGE BRUSHED STEEL ARGE BRUSHED STEEL 1 +Manufacturer#1 LARGE BURNISHED STEEL ARGE BURNISHED STEEL 2 +Manufacturer#1 PROMO BURNISHED NICKEL ROMO BURNISHED NICKEL 3 +Manufacturer#1 PROMO PLATED TIN ROMO PLATED TIN 4 +Manufacturer#1 PROMO PLATED TIN ROMO PLATED TIN 4 +Manufacturer#1 STANDARD ANODIZED STEEL TANDARD ANODIZED STEEL 6 +Manufacturer#2 ECONOMY POLISHED STEEL CONOMY POLISHED STEEL 1 +Manufacturer#2 MEDIUM ANODIZED COPPER EDIUM ANODIZED COPPER 2 +Manufacturer#2 MEDIUM BURNISHED COPPER EDIUM BURNISHED COPPER 3 +Manufacturer#2 SMALL POLISHED NICKEL MALL POLISHED NICKEL 4 +Manufacturer#2 STANDARD PLATED TIN TANDARD PLATED TIN 5 +Manufacturer#3 ECONOMY PLATED COPPER CONOMY PLATED COPPER 1 +Manufacturer#3 MEDIUM BURNISHED BRASS EDIUM BURNISHED BRASS 2 +Manufacturer#3 MEDIUM BURNISHED TIN EDIUM BURNISHED TIN 3 +Manufacturer#3 PROMO ANODIZED TIN ROMO ANODIZED TIN 4 +Manufacturer#3 STANDARD POLISHED STEEL TANDARD POLISHED STEEL 5 +Manufacturer#4 ECONOMY BRUSHED COPPER CONOMY BRUSHED COPPER 1 +Manufacturer#4 SMALL BRUSHED BRASS MALL BRUSHED BRASS 2 +Manufacturer#4 SMALL PLATED STEEL MALL PLATED STEEL 3 +Manufacturer#4 PROMO POLISHED STEEL ROMO POLISHED STEEL 4 +Manufacturer#4 STANDARD ANODIZED TIN TANDARD ANODIZED TIN 5 +Manufacturer#5 LARGE BRUSHED BRASS ARGE BRUSHED BRASS 1 +Manufacturer#5 ECONOMY BURNISHED STEEL CONOMY BURNISHED STEEL 2 +Manufacturer#5 MEDIUM BURNISHED TIN EDIUM BURNISHED TIN 3 +Manufacturer#5 SMALL PLATED BRASS MALL PLATED BRASS 4 +Manufacturer#5 STANDARD BURNISHED TIN TANDARD BURNISHED TIN 5 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 4. testCount-0-e6e97e884327df86f16b870527ec026c b/sql/hive/src/test/resources/golden/windowing.q -- 4. testCount-0-e6e97e884327df86f16b870527ec026c new file mode 100644 index 0000000000000..bf8e620a304af --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 4. testCount-0-e6e97e884327df86f16b870527ec026c @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 +Manufacturer#1 almond antique burnished rose metallic 2 +Manufacturer#1 almond antique chartreuse lavender yellow 3 +Manufacturer#1 almond antique salmon chartreuse burlywood 4 +Manufacturer#1 almond aquamarine burnished black steel 5 +Manufacturer#1 almond aquamarine pink moccasin thistle 6 +Manufacturer#2 almond antique violet chocolate turquoise 1 +Manufacturer#2 almond antique violet turquoise frosted 2 +Manufacturer#2 almond aquamarine midnight light salmon 3 +Manufacturer#2 almond aquamarine rose maroon antique 4 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 5 +Manufacturer#3 almond antique chartreuse khaki white 1 +Manufacturer#3 almond antique forest lavender goldenrod 2 +Manufacturer#3 almond antique metallic orange dim 3 +Manufacturer#3 almond antique misty red olive 4 +Manufacturer#3 almond antique olive coral navajo 5 +Manufacturer#4 almond antique gainsboro frosted violet 1 +Manufacturer#4 almond antique violet mint lemon 2 +Manufacturer#4 almond aquamarine floral ivory bisque 3 +Manufacturer#4 almond aquamarine yellow dodger mint 4 +Manufacturer#4 almond azure aquamarine papaya violet 5 +Manufacturer#5 almond antique blue firebrick mint 1 +Manufacturer#5 almond antique medium spring khaki 2 +Manufacturer#5 almond antique sky peru orange 3 +Manufacturer#5 almond aquamarine dodger light gainsboro 4 +Manufacturer#5 almond azure blanched chiffon midnight 5 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 40. testNoBetweenForRows-0-99007f45b6406869e048b0e4eb9213f1 b/sql/hive/src/test/resources/golden/windowing.q -- 40. testNoBetweenForRows-0-99007f45b6406869e048b0e4eb9213f1 new file mode 100644 index 0000000000000..1e29df62901d0 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 40. testNoBetweenForRows-0-99007f45b6406869e048b0e4eb9213f1 @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 1173.15 +Manufacturer#1 almond antique burnished rose metallic 2 2346.3 +Manufacturer#1 almond antique chartreuse lavender yellow 34 4100.06 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 5702.650000000001 +Manufacturer#1 almond aquamarine burnished black steel 28 7117.070000000001 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 8749.730000000001 +Manufacturer#2 almond antique violet chocolate turquoise 14 1690.68 +Manufacturer#2 almond antique violet turquoise frosted 40 3491.38 +Manufacturer#2 almond aquamarine midnight light salmon 2 5523.360000000001 +Manufacturer#2 almond aquamarine rose maroon antique 25 7222.02 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 8923.62 +Manufacturer#3 almond antique chartreuse khaki white 17 1671.68 +Manufacturer#3 almond antique forest lavender goldenrod 14 2861.95 +Manufacturer#3 almond antique metallic orange dim 19 4272.34 +Manufacturer#3 almond antique misty red olive 1 6195.32 +Manufacturer#3 almond antique olive coral navajo 45 7532.61 +Manufacturer#4 almond antique gainsboro frosted violet 10 1620.67 +Manufacturer#4 almond antique violet mint lemon 39 2996.09 +Manufacturer#4 almond aquamarine floral ivory bisque 27 4202.35 +Manufacturer#4 almond aquamarine yellow dodger mint 7 6047.27 +Manufacturer#4 almond azure aquamarine papaya violet 12 7337.620000000001 +Manufacturer#5 almond antique blue firebrick mint 31 1789.69 +Manufacturer#5 almond antique medium spring khaki 6 3401.3500000000004 +Manufacturer#5 almond antique sky peru orange 2 5190.08 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 6208.18 +Manufacturer#5 almond azure blanched chiffon midnight 23 7672.66 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 41. testNoBetweenForRange-0-d81a591e90950de291d2f133793e9283 b/sql/hive/src/test/resources/golden/windowing.q -- 41. testNoBetweenForRange-0-d81a591e90950de291d2f133793e9283 new file mode 100644 index 0000000000000..a620479fe4064 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 41. testNoBetweenForRange-0-d81a591e90950de291d2f133793e9283 @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 2346.3 +Manufacturer#1 almond antique burnished rose metallic 2 2346.3 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 3948.8900000000003 +Manufacturer#1 almond aquamarine burnished black steel 28 5363.31 +Manufacturer#1 almond antique chartreuse lavender yellow 34 7117.070000000001 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 8749.730000000001 +Manufacturer#2 almond aquamarine midnight light salmon 2 2031.98 +Manufacturer#2 almond antique violet chocolate turquoise 14 3722.66 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5424.26 +Manufacturer#2 almond aquamarine rose maroon antique 25 7122.92 +Manufacturer#2 almond antique violet turquoise frosted 40 8923.62 +Manufacturer#3 almond antique misty red olive 1 1922.98 +Manufacturer#3 almond antique forest lavender goldenrod 14 3113.25 +Manufacturer#3 almond antique chartreuse khaki white 17 4784.93 +Manufacturer#3 almond antique metallic orange dim 19 6195.320000000001 +Manufacturer#3 almond antique olive coral navajo 45 7532.610000000001 +Manufacturer#4 almond aquamarine yellow dodger mint 7 1844.92 +Manufacturer#4 almond antique gainsboro frosted violet 10 3465.59 +Manufacturer#4 almond azure aquamarine papaya violet 12 4755.9400000000005 +Manufacturer#4 almond aquamarine floral ivory bisque 27 5962.200000000001 +Manufacturer#4 almond antique violet mint lemon 39 7337.620000000001 +Manufacturer#5 almond antique sky peru orange 2 1788.73 +Manufacturer#5 almond antique medium spring khaki 6 3400.3900000000003 +Manufacturer#5 almond azure blanched chiffon midnight 23 4864.870000000001 +Manufacturer#5 almond antique blue firebrick mint 31 6654.560000000001 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 7672.660000000002 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 42. testUnboundedFollowingForRows-0-fb8648e82e4dd56d6bdcfd739dd1edf0 b/sql/hive/src/test/resources/golden/windowing.q -- 42. testUnboundedFollowingForRows-0-fb8648e82e4dd56d6bdcfd739dd1edf0 new file mode 100644 index 0000000000000..74147d2571a15 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 42. testUnboundedFollowingForRows-0-fb8648e82e4dd56d6bdcfd739dd1edf0 @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 8749.730000000001 +Manufacturer#1 almond antique burnished rose metallic 2 7576.58 +Manufacturer#1 almond antique chartreuse lavender yellow 34 6403.43 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 4649.67 +Manufacturer#1 almond aquamarine burnished black steel 28 3047.08 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 1632.66 +Manufacturer#2 almond antique violet chocolate turquoise 14 8923.62 +Manufacturer#2 almond antique violet turquoise frosted 40 7232.9400000000005 +Manufacturer#2 almond aquamarine midnight light salmon 2 5432.24 +Manufacturer#2 almond aquamarine rose maroon antique 25 3400.26 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 1701.6 +Manufacturer#3 almond antique chartreuse khaki white 17 7532.61 +Manufacturer#3 almond antique forest lavender goldenrod 14 5860.929999999999 +Manufacturer#3 almond antique metallic orange dim 19 4670.66 +Manufacturer#3 almond antique misty red olive 1 3260.27 +Manufacturer#3 almond antique olive coral navajo 45 1337.29 +Manufacturer#4 almond antique gainsboro frosted violet 10 7337.620000000001 +Manufacturer#4 almond antique violet mint lemon 39 5716.950000000001 +Manufacturer#4 almond aquamarine floral ivory bisque 27 4341.530000000001 +Manufacturer#4 almond aquamarine yellow dodger mint 7 3135.27 +Manufacturer#4 almond azure aquamarine papaya violet 12 1290.35 +Manufacturer#5 almond antique blue firebrick mint 31 7672.66 +Manufacturer#5 almond antique medium spring khaki 6 5882.970000000001 +Manufacturer#5 almond antique sky peru orange 2 4271.3099999999995 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 2482.58 +Manufacturer#5 almond azure blanched chiffon midnight 23 1464.48 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 43. testUnboundedFollowingForRange-0-3cd04e5f2398853c4850f4f86142bb39 b/sql/hive/src/test/resources/golden/windowing.q -- 43. testUnboundedFollowingForRange-0-3cd04e5f2398853c4850f4f86142bb39 new file mode 100644 index 0000000000000..49d003b5de139 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 43. testUnboundedFollowingForRange-0-3cd04e5f2398853c4850f4f86142bb39 @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 8749.730000000001 +Manufacturer#1 almond antique burnished rose metallic 2 8749.730000000001 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 6403.43 +Manufacturer#1 almond aquamarine burnished black steel 28 4800.84 +Manufacturer#1 almond antique chartreuse lavender yellow 34 3386.42 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 1632.66 +Manufacturer#2 almond aquamarine midnight light salmon 2 8923.62 +Manufacturer#2 almond antique violet chocolate turquoise 14 6891.639999999999 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5200.96 +Manufacturer#2 almond aquamarine rose maroon antique 25 3499.36 +Manufacturer#2 almond antique violet turquoise frosted 40 1800.7 +Manufacturer#3 almond antique misty red olive 1 7532.610000000001 +Manufacturer#3 almond antique forest lavender goldenrod 14 5609.63 +Manufacturer#3 almond antique chartreuse khaki white 17 4419.360000000001 +Manufacturer#3 almond antique metallic orange dim 19 2747.6800000000003 +Manufacturer#3 almond antique olive coral navajo 45 1337.29 +Manufacturer#4 almond aquamarine yellow dodger mint 7 7337.620000000001 +Manufacturer#4 almond antique gainsboro frosted violet 10 5492.7 +Manufacturer#4 almond azure aquamarine papaya violet 12 3872.0299999999997 +Manufacturer#4 almond aquamarine floral ivory bisque 27 2581.6800000000003 +Manufacturer#4 almond antique violet mint lemon 39 1375.42 +Manufacturer#5 almond antique sky peru orange 2 7672.660000000002 +Manufacturer#5 almond antique medium spring khaki 6 5883.93 +Manufacturer#5 almond azure blanched chiffon midnight 23 4272.27 +Manufacturer#5 almond antique blue firebrick mint 31 2807.79 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 1018.1 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 44. testOverNoPartitionSingleAggregate-0-cb3d2f8c1296044dc2658876bb6103ae b/sql/hive/src/test/resources/golden/windowing.q -- 44. testOverNoPartitionSingleAggregate-0-cb3d2f8c1296044dc2658876bb6103ae new file mode 100644 index 0000000000000..5982c9ee2a4d4 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 44. testOverNoPartitionSingleAggregate-0-cb3d2f8c1296044dc2658876bb6103ae @@ -0,0 +1,26 @@ +almond antique blue firebrick mint 1789.69 1546.78 +almond antique burnished rose metallic 1173.15 1546.78 +almond antique burnished rose metallic 1173.15 1546.78 +almond antique chartreuse khaki white 1671.68 1546.78 +almond antique chartreuse lavender yellow 1753.76 1546.78 +almond antique forest lavender goldenrod 1190.27 1546.78 +almond antique gainsboro frosted violet 1620.67 1546.78 +almond antique medium spring khaki 1611.66 1546.78 +almond antique metallic orange dim 1410.39 1546.78 +almond antique misty red olive 1922.98 1546.78 +almond antique olive coral navajo 1337.29 1546.78 +almond antique salmon chartreuse burlywood 1602.59 1546.78 +almond antique sky peru orange 1788.73 1546.78 +almond antique violet chocolate turquoise 1690.68 1546.78 +almond antique violet mint lemon 1375.42 1546.78 +almond antique violet turquoise frosted 1800.7 1546.78 +almond aquamarine burnished black steel 1414.42 1546.78 +almond aquamarine dodger light gainsboro 1018.1 1546.78 +almond aquamarine floral ivory bisque 1206.26 1546.78 +almond aquamarine midnight light salmon 2031.98 1546.78 +almond aquamarine pink moccasin thistle 1632.66 1546.78 +almond aquamarine rose maroon antique 1698.66 1546.78 +almond aquamarine sandy cyan gainsboro 1701.6 1546.78 +almond aquamarine yellow dodger mint 1844.92 1546.78 +almond azure aquamarine papaya violet 1290.35 1546.78 +almond azure blanched chiffon midnight 1464.48 1546.78 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 5. testCountWithWindowingUDAF-0-3bde93728761b780a745c2ce0398aa0f b/sql/hive/src/test/resources/golden/windowing.q -- 5. testCountWithWindowingUDAF-0-3bde93728761b780a745c2ce0398aa0f new file mode 100644 index 0000000000000..00d41fc0bcd9c --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 5. testCountWithWindowingUDAF-0-3bde93728761b780a745c2ce0398aa0f @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 1 1 2 1173.15 1173.15 2 0 +Manufacturer#1 almond antique burnished rose metallic 1 1 2 1173.15 2346.3 2 0 +Manufacturer#1 almond antique chartreuse lavender yellow 3 2 3 1753.76 4100.06 34 32 +Manufacturer#1 almond antique salmon chartreuse burlywood 4 3 4 1602.59 5702.650000000001 6 -28 +Manufacturer#1 almond aquamarine burnished black steel 5 4 5 1414.42 7117.070000000001 28 22 +Manufacturer#1 almond aquamarine pink moccasin thistle 6 5 6 1632.66 8749.730000000001 42 14 +Manufacturer#2 almond antique violet chocolate turquoise 1 1 1 1690.68 1690.68 14 0 +Manufacturer#2 almond antique violet turquoise frosted 2 2 2 1800.7 3491.38 40 26 +Manufacturer#2 almond aquamarine midnight light salmon 3 3 3 2031.98 5523.360000000001 2 -38 +Manufacturer#2 almond aquamarine rose maroon antique 4 4 4 1698.66 7222.02 25 23 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 5 5 5 1701.6 8923.62 18 -7 +Manufacturer#3 almond antique chartreuse khaki white 1 1 1 1671.68 1671.68 17 0 +Manufacturer#3 almond antique forest lavender goldenrod 2 2 2 1190.27 2861.95 14 -3 +Manufacturer#3 almond antique metallic orange dim 3 3 3 1410.39 4272.34 19 5 +Manufacturer#3 almond antique misty red olive 4 4 4 1922.98 6195.32 1 -18 +Manufacturer#3 almond antique olive coral navajo 5 5 5 1337.29 7532.61 45 44 +Manufacturer#4 almond antique gainsboro frosted violet 1 1 1 1620.67 1620.67 10 0 +Manufacturer#4 almond antique violet mint lemon 2 2 2 1375.42 2996.09 39 29 +Manufacturer#4 almond aquamarine floral ivory bisque 3 3 3 1206.26 4202.35 27 -12 +Manufacturer#4 almond aquamarine yellow dodger mint 4 4 4 1844.92 6047.27 7 -20 +Manufacturer#4 almond azure aquamarine papaya violet 5 5 5 1290.35 7337.620000000001 12 5 +Manufacturer#5 almond antique blue firebrick mint 1 1 1 1789.69 1789.69 31 0 +Manufacturer#5 almond antique medium spring khaki 2 2 2 1611.66 3401.3500000000004 6 -25 +Manufacturer#5 almond antique sky peru orange 3 3 3 1788.73 5190.08 2 -4 +Manufacturer#5 almond aquamarine dodger light gainsboro 4 4 4 1018.1 6208.18 46 44 +Manufacturer#5 almond azure blanched chiffon midnight 5 5 5 1464.48 7672.66 23 -23 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 6. testCountInSubQ-0-73d5274a21d4f4fd51d2a0f1d98516ce b/sql/hive/src/test/resources/golden/windowing.q -- 6. testCountInSubQ-0-73d5274a21d4f4fd51d2a0f1d98516ce new file mode 100644 index 0000000000000..98c09e4fe15c3 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 6. testCountInSubQ-0-73d5274a21d4f4fd51d2a0f1d98516ce @@ -0,0 +1,26 @@ +1 1 2 1173.15 0 +1 1 2 2346.3 0 +3 2 3 4100.06 32 +4 3 4 5702.650000000001 -28 +5 4 5 7117.070000000001 22 +6 5 6 8749.730000000001 14 +1 1 1 1690.68 0 +2 2 2 3491.38 26 +3 3 3 5523.360000000001 -38 +4 4 4 7222.02 23 +5 5 5 8923.62 -7 +1 1 1 1671.68 0 +2 2 2 2861.95 -3 +3 3 3 4272.34 5 +4 4 4 6195.32 -18 +5 5 5 7532.61 44 +1 1 1 1620.67 0 +2 2 2 2996.09 29 +3 3 3 4202.35 -12 +4 4 4 6047.27 -20 +5 5 5 7337.620000000001 5 +1 1 1 1789.69 0 +2 2 2 3401.3500000000004 -25 +3 3 3 5190.08 -4 +4 4 4 6208.18 44 +5 5 5 7672.66 -23 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 8. testMixedCaseAlias-0-4b1ad2515fb079012467e987f484a722 b/sql/hive/src/test/resources/golden/windowing.q -- 8. testMixedCaseAlias-0-4b1ad2515fb079012467e987f484a722 new file mode 100644 index 0000000000000..e35257d98382e --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 8. testMixedCaseAlias-0-4b1ad2515fb079012467e987f484a722 @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 1 +Manufacturer#1 almond antique burnished rose metallic 2 1 +Manufacturer#1 almond antique chartreuse lavender yellow 34 3 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 4 +Manufacturer#1 almond aquamarine burnished black steel 28 5 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 6 +Manufacturer#2 almond antique violet chocolate turquoise 14 1 +Manufacturer#2 almond antique violet turquoise frosted 40 2 +Manufacturer#2 almond aquamarine midnight light salmon 2 3 +Manufacturer#2 almond aquamarine rose maroon antique 25 4 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5 +Manufacturer#3 almond antique chartreuse khaki white 17 1 +Manufacturer#3 almond antique forest lavender goldenrod 14 2 +Manufacturer#3 almond antique metallic orange dim 19 3 +Manufacturer#3 almond antique misty red olive 1 4 +Manufacturer#3 almond antique olive coral navajo 45 5 +Manufacturer#4 almond antique gainsboro frosted violet 10 1 +Manufacturer#4 almond antique violet mint lemon 39 2 +Manufacturer#4 almond aquamarine floral ivory bisque 27 3 +Manufacturer#4 almond aquamarine yellow dodger mint 7 4 +Manufacturer#4 almond azure aquamarine papaya violet 12 5 +Manufacturer#5 almond antique blue firebrick mint 31 1 +Manufacturer#5 almond antique medium spring khaki 6 2 +Manufacturer#5 almond antique sky peru orange 2 3 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 4 +Manufacturer#5 almond azure blanched chiffon midnight 23 5 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 9. testHavingWithWindowingNoGBY-0-70cdc0555a61ef08534a9ebebb95ebbf b/sql/hive/src/test/resources/golden/windowing.q -- 9. testHavingWithWindowingNoGBY-0-70cdc0555a61ef08534a9ebebb95ebbf new file mode 100644 index 0000000000000..850c41c8115d6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 9. testHavingWithWindowingNoGBY-0-70cdc0555a61ef08534a9ebebb95ebbf @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 1 1 1173.15 +Manufacturer#1 almond antique burnished rose metallic 2 1 1 2346.3 +Manufacturer#1 almond antique chartreuse lavender yellow 34 3 2 4100.06 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 4 3 5702.650000000001 +Manufacturer#1 almond aquamarine burnished black steel 28 5 4 7117.070000000001 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 6 5 8749.730000000001 +Manufacturer#2 almond antique violet chocolate turquoise 14 1 1 1690.68 +Manufacturer#2 almond antique violet turquoise frosted 40 2 2 3491.38 +Manufacturer#2 almond aquamarine midnight light salmon 2 3 3 5523.360000000001 +Manufacturer#2 almond aquamarine rose maroon antique 25 4 4 7222.02 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5 5 8923.62 +Manufacturer#3 almond antique chartreuse khaki white 17 1 1 1671.68 +Manufacturer#3 almond antique forest lavender goldenrod 14 2 2 2861.95 +Manufacturer#3 almond antique metallic orange dim 19 3 3 4272.34 +Manufacturer#3 almond antique misty red olive 1 4 4 6195.32 +Manufacturer#3 almond antique olive coral navajo 45 5 5 7532.61 +Manufacturer#4 almond antique gainsboro frosted violet 10 1 1 1620.67 +Manufacturer#4 almond antique violet mint lemon 39 2 2 2996.09 +Manufacturer#4 almond aquamarine floral ivory bisque 27 3 3 4202.35 +Manufacturer#4 almond aquamarine yellow dodger mint 7 4 4 6047.27 +Manufacturer#4 almond azure aquamarine papaya violet 12 5 5 7337.620000000001 +Manufacturer#5 almond antique blue firebrick mint 31 1 1 1789.69 +Manufacturer#5 almond antique medium spring khaki 6 2 2 3401.3500000000004 +Manufacturer#5 almond antique sky peru orange 2 3 3 5190.08 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 4 4 6208.18 +Manufacturer#5 almond azure blanched chiffon midnight 23 5 5 7672.66 diff --git a/sql/hive/src/test/resources/golden/windowing_adjust_rowcontainer_sz-0-d3f50875bd5dff172cf813fdb7d738eb b/sql/hive/src/test/resources/golden/windowing_adjust_rowcontainer_sz-0-d3f50875bd5dff172cf813fdb7d738eb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/windowing_adjust_rowcontainer_sz-1-dda16565b98926fc3587de937b9401c7 b/sql/hive/src/test/resources/golden/windowing_adjust_rowcontainer_sz-1-dda16565b98926fc3587de937b9401c7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/windowing_adjust_rowcontainer_sz-2-374e39786feb745cd70f25be58bfa24 b/sql/hive/src/test/resources/golden/windowing_adjust_rowcontainer_sz-2-374e39786feb745cd70f25be58bfa24 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/windowing_adjust_rowcontainer_sz-3-d2b5e23edec42a62e61750b110ecbaac b/sql/hive/src/test/resources/golden/windowing_adjust_rowcontainer_sz-3-d2b5e23edec42a62e61750b110ecbaac new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_adjust_rowcontainer_sz-3-d2b5e23edec42a62e61750b110ecbaac @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/windowing_adjust_rowcontainer_sz-4-50d0c630159068b5b8ccdeb76493f1f7 b/sql/hive/src/test/resources/golden/windowing_adjust_rowcontainer_sz-4-50d0c630159068b5b8ccdeb76493f1f7 new file mode 100644 index 0000000000000..850c41c8115d6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_adjust_rowcontainer_sz-4-50d0c630159068b5b8ccdeb76493f1f7 @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 1 1 1173.15 +Manufacturer#1 almond antique burnished rose metallic 2 1 1 2346.3 +Manufacturer#1 almond antique chartreuse lavender yellow 34 3 2 4100.06 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 4 3 5702.650000000001 +Manufacturer#1 almond aquamarine burnished black steel 28 5 4 7117.070000000001 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 6 5 8749.730000000001 +Manufacturer#2 almond antique violet chocolate turquoise 14 1 1 1690.68 +Manufacturer#2 almond antique violet turquoise frosted 40 2 2 3491.38 +Manufacturer#2 almond aquamarine midnight light salmon 2 3 3 5523.360000000001 +Manufacturer#2 almond aquamarine rose maroon antique 25 4 4 7222.02 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5 5 8923.62 +Manufacturer#3 almond antique chartreuse khaki white 17 1 1 1671.68 +Manufacturer#3 almond antique forest lavender goldenrod 14 2 2 2861.95 +Manufacturer#3 almond antique metallic orange dim 19 3 3 4272.34 +Manufacturer#3 almond antique misty red olive 1 4 4 6195.32 +Manufacturer#3 almond antique olive coral navajo 45 5 5 7532.61 +Manufacturer#4 almond antique gainsboro frosted violet 10 1 1 1620.67 +Manufacturer#4 almond antique violet mint lemon 39 2 2 2996.09 +Manufacturer#4 almond aquamarine floral ivory bisque 27 3 3 4202.35 +Manufacturer#4 almond aquamarine yellow dodger mint 7 4 4 6047.27 +Manufacturer#4 almond azure aquamarine papaya violet 12 5 5 7337.620000000001 +Manufacturer#5 almond antique blue firebrick mint 31 1 1 1789.69 +Manufacturer#5 almond antique medium spring khaki 6 2 2 3401.3500000000004 +Manufacturer#5 almond antique sky peru orange 2 3 3 5190.08 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 4 4 6208.18 +Manufacturer#5 almond azure blanched chiffon midnight 23 5 5 7672.66 diff --git a/sql/hive/src/test/resources/golden/windowing_adjust_rowcontainer_sz-5-3f95cd6f4add7a2d0101fe3dd97e5082 b/sql/hive/src/test/resources/golden/windowing_adjust_rowcontainer_sz-5-3f95cd6f4add7a2d0101fe3dd97e5082 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_adjust_rowcontainer_sz-5-3f95cd6f4add7a2d0101fe3dd97e5082 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/windowing_columnPruning-0-d3f50875bd5dff172cf813fdb7d738eb b/sql/hive/src/test/resources/golden/windowing_columnPruning-0-d3f50875bd5dff172cf813fdb7d738eb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/windowing_columnPruning-1-dda16565b98926fc3587de937b9401c7 b/sql/hive/src/test/resources/golden/windowing_columnPruning-1-dda16565b98926fc3587de937b9401c7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/windowing_columnPruning-2-374e39786feb745cd70f25be58bfa24 b/sql/hive/src/test/resources/golden/windowing_columnPruning-2-374e39786feb745cd70f25be58bfa24 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/windowing_columnPruning-3-9294b4a22bc396ff2accabd53c5da98b b/sql/hive/src/test/resources/golden/windowing_columnPruning-3-9294b4a22bc396ff2accabd53c5da98b new file mode 100644 index 0000000000000..1b5ae55383a4a --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_columnPruning-3-9294b4a22bc396ff2accabd53c5da98b @@ -0,0 +1,26 @@ +2 1173.15 +2 2346.3 +34 4100.06 +6 5702.650000000001 +28 7117.070000000001 +42 8749.730000000001 +14 1690.68 +40 3491.38 +2 5523.360000000001 +25 7222.02 +18 8923.62 +17 1671.68 +14 2861.95 +19 4272.34 +1 6195.32 +45 7532.61 +10 1620.67 +39 2996.09 +27 4202.35 +7 6047.27 +12 7337.620000000001 +31 1789.69 +6 3401.3500000000004 +2 5190.08 +46 6208.18 +23 7672.66 diff --git a/sql/hive/src/test/resources/golden/windowing_columnPruning-4-445cab062581c449ceffcb368cdf133 b/sql/hive/src/test/resources/golden/windowing_columnPruning-4-445cab062581c449ceffcb368cdf133 new file mode 100644 index 0000000000000..1b5ae55383a4a --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_columnPruning-4-445cab062581c449ceffcb368cdf133 @@ -0,0 +1,26 @@ +2 1173.15 +2 2346.3 +34 4100.06 +6 5702.650000000001 +28 7117.070000000001 +42 8749.730000000001 +14 1690.68 +40 3491.38 +2 5523.360000000001 +25 7222.02 +18 8923.62 +17 1671.68 +14 2861.95 +19 4272.34 +1 6195.32 +45 7532.61 +10 1620.67 +39 2996.09 +27 4202.35 +7 6047.27 +12 7337.620000000001 +31 1789.69 +6 3401.3500000000004 +2 5190.08 +46 6208.18 +23 7672.66 diff --git a/sql/hive/src/test/resources/golden/windowing_columnPruning-5-89110070c761eafb992eb9315128b53f b/sql/hive/src/test/resources/golden/windowing_columnPruning-5-89110070c761eafb992eb9315128b53f new file mode 100644 index 0000000000000..e426c725b0e36 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_columnPruning-5-89110070c761eafb992eb9315128b53f @@ -0,0 +1,26 @@ +Manufacturer#1 1173.15 +Manufacturer#1 2346.3 +Manufacturer#1 4100.06 +Manufacturer#1 5702.650000000001 +Manufacturer#1 7117.070000000001 +Manufacturer#1 8749.730000000001 +Manufacturer#2 1690.68 +Manufacturer#2 3491.38 +Manufacturer#2 5523.360000000001 +Manufacturer#2 7222.02 +Manufacturer#2 8923.62 +Manufacturer#3 1671.68 +Manufacturer#3 2861.95 +Manufacturer#3 4272.34 +Manufacturer#3 6195.32 +Manufacturer#3 7532.61 +Manufacturer#4 1620.67 +Manufacturer#4 2996.09 +Manufacturer#4 4202.35 +Manufacturer#4 6047.27 +Manufacturer#4 7337.620000000001 +Manufacturer#5 1789.69 +Manufacturer#5 3401.3500000000004 +Manufacturer#5 5190.08 +Manufacturer#5 6208.18 +Manufacturer#5 7672.66 diff --git a/sql/hive/src/test/resources/golden/windowing_multipartitioning.q (deterministic) 1-0-12a92d8800e0da8b515ba3eaf6a7fd0f b/sql/hive/src/test/resources/golden/windowing_multipartitioning.q (deterministic) 1-0-12a92d8800e0da8b515ba3eaf6a7fd0f new file mode 100644 index 0000000000000..acc4f3bc2a2d5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_multipartitioning.q (deterministic) 1-0-12a92d8800e0da8b515ba3eaf6a7fd0f @@ -0,0 +1,1049 @@ + 1 4294967354 + 2 8589934811 + 3 12884902227 +alice allen 1 4294967487 +alice allen 2 8589935012 +alice allen 3 12884902543 +alice brown 1 4294967355 +alice carson 1 4294967370 +alice davidson 1 4294967517 +alice falkner 1 4294967316 +alice garcia 1 4294967369 +alice hernandez 1 4294967314 +alice hernandez 2 8589934613 +alice johnson 1 4294967424 +alice king 1 4294967387 +alice king 2 8589934903 +alice king 3 12884902449 +alice laertes 1 4294967519 +alice laertes 2 8589935038 +alice miller 1 4294967324 +alice nixon 1 4294967484 +alice nixon 2 8589934894 +alice nixon 3 12884902307 +alice ovid 1 4294967412 +alice polk 1 4294967366 +alice quirinius 1 4294967505 +alice quirinius 2 8589935054 +alice robinson 1 4294967502 +alice robinson 2 8589934947 +alice steinbeck 1 4294967549 +alice steinbeck 2 8589934913 +alice steinbeck 3 12884902387 +alice underhill 1 4294967441 +alice van buren 1 4294967428 +alice xylophone 1 4294967519 +alice xylophone 2 8589934877 +alice xylophone 3 12884902240 +alice zipper 1 4294967380 +alice zipper 2 8589934919 +alice zipper 3 12884902439 +bob brown 1 4294967431 +bob brown 2 8589934853 +bob brown 3 12884902280 +bob carson 1 4294967408 +bob davidson 1 4294967435 +bob davidson 2 8589934939 +bob davidson 3 12884902293 +bob ellison 1 4294967530 +bob ellison 2 8589934966 +bob ellison 3 12884902328 +bob ellison 4 17179869672 +bob falkner 1 4294967464 +bob garcia 1 4294967435 +bob garcia 2 8589934804 +bob garcia 3 12884902148 +bob garcia 4 17179869587 +bob garcia 5 21474836905 +bob hernandez 1 4294967500 +bob ichabod 1 4294967424 +bob king 1 4294967443 +bob king 2 8589934740 +bob king 3 12884902279 +bob laertes 1 4294967472 +bob laertes 2 8589934852 +bob miller 1 4294967349 +bob ovid 1 4294967401 +bob ovid 2 8589934801 +bob ovid 3 12884902313 +bob ovid 4 17179869708 +bob polk 1 4294967337 +bob quirinius 1 4294967346 +bob steinbeck 1 4294967342 +bob van buren 1 4294967422 +bob white 1 4294967493 +bob white 2 8589934855 +bob xylophone 1 4294967407 +bob xylophone 2 8589934872 +bob young 1 4294967413 +bob zipper 1 4294967416 +bob zipper 2 8589934769 +bob zipper 3 12884902068 +calvin allen 1 4294967373 +calvin brown 1 4294967530 +calvin brown 2 8589934967 +calvin brown 3 12884902378 +calvin carson 1 4294967534 +calvin davidson 1 4294967437 +calvin davidson 2 8589934905 +calvin ellison 1 4294967480 +calvin falkner 1 4294967305 +calvin falkner 2 8589934723 +calvin falkner 3 12884902155 +calvin falkner 4 17179869455 +calvin falkner 5 21474836800 +calvin falkner 6 25769804250 +calvin garcia 1 4294967492 +calvin hernandez 1 4294967341 +calvin johnson 1 4294967546 +calvin laertes 1 4294967499 +calvin laertes 2 8589934930 +calvin nixon 1 4294967488 +calvin nixon 2 8589934788 +calvin nixon 3 12884902200 +calvin ovid 1 4294967343 +calvin ovid 2 8589934881 +calvin ovid 3 12884902210 +calvin ovid 4 17179869559 +calvin polk 1 4294967416 +calvin quirinius 1 4294967532 +calvin quirinius 2 8589935053 +calvin robinson 1 4294967326 +calvin steinbeck 1 4294967417 +calvin steinbeck 2 8589934891 +calvin steinbeck 3 12884902396 +calvin thompson 1 4294967346 +calvin thompson 2 8589934859 +calvin underhill 1 4294967478 +calvin van buren 1 4294967300 +calvin van buren 2 8589934808 +calvin white 1 4294967304 +calvin white 2 8589934848 +calvin xylophone 1 4294967299 +calvin xylophone 2 8589934675 +calvin xylophone 3 12884902133 +calvin young 1 4294967410 +calvin young 2 8589934752 +calvin zipper 1 4294967441 +calvin zipper 2 8589934960 +david allen 1 4294967381 +david allen 2 8589934752 +david brown 1 4294967544 +david brown 2 8589934870 +david davidson 1 4294967487 +david davidson 2 8589934952 +david davidson 3 12884902474 +david davidson 4 17179869819 +david ellison 1 4294967477 +david ellison 2 8589934963 +david ellison 3 12884902426 +david hernandez 1 4294967324 +david ichabod 1 4294967385 +david ichabod 2 8589934872 +david laertes 1 4294967385 +david nixon 1 4294967381 +david ovid 1 4294967396 +david ovid 2 8589934839 +david quirinius 1 4294967375 +david quirinius 2 8589934905 +david quirinius 3 12884902362 +david robinson 1 4294967465 +david robinson 2 8589934933 +david thompson 1 4294967361 +david underhill 1 4294967331 +david underhill 2 8589934715 +david underhill 3 12884902185 +david van buren 1 4294967438 +david van buren 2 8589934747 +david white 1 4294967428 +david xylophone 1 8589934898 +david xylophone 1 8589934898 +david xylophone 3 12884902378 +david young 1 4294967296 +david young 2 8589934601 +ethan allen 1 4294967351 +ethan brown 1 4294967477 +ethan brown 2 8589934897 +ethan brown 3 12884902217 +ethan brown 4 17179869548 +ethan brown 5 21474836951 +ethan brown 6 25769804375 +ethan carson 1 4294967352 +ethan ellison 1 4294967514 +ethan ellison 2 8589934923 +ethan falkner 1 4294967318 +ethan falkner 2 8589934779 +ethan garcia 1 4294967310 +ethan hernandez 1 4294967349 +ethan johnson 1 4294967357 +ethan king 1 4294967413 +ethan laertes 1 4294967402 +ethan laertes 2 8589934859 +ethan laertes 3 12884902390 +ethan laertes 4 17179869880 +ethan laertes 5 21474837302 +ethan laertes 6 25769804603 +ethan laertes 7 30064771974 +ethan miller 1 4294967352 +ethan nixon 1 4294967499 +ethan ovid 1 4294967452 +ethan polk 1 4294967329 +ethan polk 2 8589934711 +ethan polk 3 12884902253 +ethan polk 4 17179869732 +ethan quirinius 1 4294967501 +ethan quirinius 2 8589934852 +ethan quirinius 3 12884902200 +ethan robinson 1 4294967353 +ethan robinson 2 8589934855 +ethan underhill 1 4294967466 +ethan van buren 1 4294967511 +ethan white 1 4294967445 +ethan white 2 8589934872 +ethan xylophone 1 4294967543 +ethan zipper 1 4294967462 +ethan zipper 2 8589934815 +fred davidson 1 4294967512 +fred davidson 2 8589934936 +fred davidson 3 12884902424 +fred ellison 1 4294967470 +fred ellison 2 8589934901 +fred ellison 3 12884902294 +fred falkner 1 4294967340 +fred falkner 2 8589934887 +fred falkner 3 12884902187 +fred hernandez 1 4294967365 +fred ichabod 1 4294967342 +fred ichabod 2 8589934831 +fred johnson 1 4294967373 +fred king 1 4294967346 +fred king 2 8589934766 +fred laertes 1 4294967351 +fred miller 1 4294967490 +fred nixon 1 4294967514 +fred nixon 2 8589934811 +fred nixon 3 12884902293 +fred nixon 4 17179869668 +fred polk 1 4294967332 +fred polk 2 8589934775 +fred polk 3 12884902233 +fred polk 4 17179869740 +fred quirinius 1 4294967426 +fred quirinius 2 8589934951 +fred robinson 1 4294967461 +fred steinbeck 1 4294967411 +fred steinbeck 2 8589934740 +fred steinbeck 3 12884902212 +fred underhill 1 4294967387 +fred van buren 1 4294967431 +fred van buren 2 8589934812 +fred van buren 3 12884902338 +fred van buren 4 17179869801 +fred white 1 4294967434 +fred young 1 4294967495 +fred young 2 8589934980 +fred zipper 1 4294967447 +gabriella allen 1 4294967405 +gabriella allen 2 8589934939 +gabriella brown 1 4294967543 +gabriella brown 2 8589934946 +gabriella carson 1 4294967540 +gabriella davidson 1 4294967507 +gabriella ellison 1 4294967393 +gabriella ellison 2 8589934733 +gabriella falkner 1 4294967378 +gabriella falkner 2 8589934901 +gabriella falkner 3 12884902335 +gabriella garcia 1 4294967419 +gabriella hernandez 1 4294967481 +gabriella hernandez 2 8589934943 +gabriella ichabod 1 4294967337 +gabriella ichabod 2 8589934725 +gabriella ichabod 3 12884902062 +gabriella ichabod 4 17179869382 +gabriella ichabod 5 21474836880 +gabriella king 1 4294967434 +gabriella king 2 8589934827 +gabriella laertes 1 4294967410 +gabriella miller 1 4294967363 +gabriella ovid 1 4294967482 +gabriella ovid 2 8589935004 +gabriella polk 1 4294967410 +gabriella polk 2 8589934712 +gabriella steinbeck 1 4294967500 +gabriella steinbeck 2 8589934935 +gabriella thompson 1 4294967299 +gabriella thompson 2 8589934711 +gabriella thompson 3 12884902196 +gabriella van buren 1 4294967457 +gabriella van buren 2 8589934927 +gabriella white 1 4294967335 +gabriella young 1 4294967493 +gabriella young 2 8589934924 +gabriella zipper 1 4294967357 +gabriella zipper 2 8589934867 +holly allen 1 4294967327 +holly brown 1 4294967321 +holly brown 2 8589934659 +holly falkner 1 4294967324 +holly hernandez 1 4294967378 +holly hernandez 2 8589934921 +holly hernandez 3 12884902465 +holly hernandez 4 17179869773 +holly ichabod 1 4294967342 +holly ichabod 2 8589934800 +holly ichabod 3 12884902129 +holly johnson 1 4294967517 +holly johnson 2 8589934897 +holly johnson 3 12884902432 +holly king 1 4294967392 +holly king 2 8589934753 +holly laertes 1 4294967406 +holly miller 1 4294967388 +holly nixon 1 4294967383 +holly nixon 2 8589934707 +holly polk 1 4294967398 +holly polk 2 8589934832 +holly robinson 1 4294967532 +holly thompson 1 4294967529 +holly thompson 2 8589934868 +holly thompson 3 12884902242 +holly underhill 1 4294967383 +holly underhill 2 8589934894 +holly underhill 3 12884902330 +holly underhill 4 17179869856 +holly van buren 1 4294967539 +holly white 1 4294967320 +holly white 2 8589934735 +holly xylophone 1 4294967435 +holly young 1 4294967487 +holly young 2 8589934987 +holly zipper 1 4294967337 +holly zipper 2 8589934846 +irene allen 1 4294967518 +irene brown 1 4294967434 +irene brown 2 8589934862 +irene brown 3 12884902272 +irene carson 1 4294967473 +irene ellison 1 4294967379 +irene ellison 2 8589934797 +irene falkner 1 4294967404 +irene falkner 2 8589934952 +irene garcia 1 4294967498 +irene garcia 2 8589934869 +irene garcia 3 12884902192 +irene ichabod 1 4294967529 +irene ichabod 2 8589935038 +irene johnson 1 4294967468 +irene laertes 1 4294967481 +irene laertes 2 8589934780 +irene laertes 3 12884902116 +irene miller 1 4294967387 +irene nixon 1 4294967323 +irene nixon 2 8589934824 +irene nixon 3 12884902362 +irene ovid 1 4294967499 +irene ovid 2 8589934870 +irene ovid 3 12884902230 +irene polk 1 4294967521 +irene polk 2 8589934930 +irene polk 3 12884902395 +irene polk 4 17179869941 +irene polk 5 21474837237 +irene quirinius 1 4294967365 +irene quirinius 2 8589934751 +irene quirinius 3 12884902141 +irene robinson 1 4294967347 +irene steinbeck 1 4294967549 +irene thompson 1 4294967479 +irene underhill 1 4294967371 +irene underhill 2 8589934753 +irene van buren 1 4294967439 +irene van buren 2 8589934906 +irene xylophone 1 4294967436 +jessica brown 1 4294967496 +jessica carson 1 4294967389 +jessica carson 2 8589934897 +jessica carson 3 12884902345 +jessica davidson 1 4294967325 +jessica davidson 2 8589934709 +jessica davidson 3 12884902098 +jessica davidson 4 17179869569 +jessica ellison 1 4294967316 +jessica ellison 2 8589934721 +jessica falkner 1 4294967549 +jessica garcia 1 4294967540 +jessica garcia 2 8589935041 +jessica ichabod 1 4294967413 +jessica johnson 1 4294967497 +jessica johnson 2 8589934870 +jessica miller 1 4294967495 +jessica nixon 1 4294967311 +jessica nixon 2 8589934754 +jessica ovid 1 4294967330 +jessica ovid 2 8589934855 +jessica polk 1 4294967403 +jessica quirinius 1 4294967523 +jessica quirinius 2 8589934942 +jessica quirinius 3 12884902388 +jessica quirinius 4 17179869696 +jessica robinson 1 4294967542 +jessica thompson 1 4294967449 +jessica thompson 2 8589934763 +jessica underhill 1 4294967541 +jessica underhill 2 8589934844 +jessica underhill 3 12884902153 +jessica van buren 1 4294967344 +jessica white 1 4294967482 +jessica white 2 8589934929 +jessica white 3 12884902378 +jessica white 4 17179869687 +jessica white 5 21474837086 +jessica xylophone 1 4294967421 +jessica young 1 4294967382 +jessica young 2 8589934903 +jessica zipper 1 4294967334 +jessica zipper 2 8589934785 +jessica zipper 3 12884902157 +katie allen 1 4294967443 +katie brown 1 4294967420 +katie davidson 1 4294967459 +katie ellison 1 4294967486 +katie ellison 2 8589934861 +katie falkner 1 4294967362 +katie garcia 1 4294967306 +katie garcia 2 8589934680 +katie hernandez 1 4294967451 +katie ichabod 1 4294967330 +katie ichabod 2 8589934742 +katie ichabod 3 12884902209 +katie king 1 4294967339 +katie king 2 8589934760 +katie king 3 12884902199 +katie miller 1 4294967425 +katie miller 2 8589934859 +katie nixon 1 4294967500 +katie ovid 1 4294967519 +katie polk 1 4294967384 +katie polk 2 8589934926 +katie robinson 1 4294967310 +katie van buren 1 4294967335 +katie van buren 2 8589934647 +katie white 1 4294967337 +katie white 2 8589934643 +katie xylophone 1 4294967486 +katie young 1 4294967349 +katie young 2 8589934681 +katie young 3 12884902225 +katie zipper 1 4294967354 +katie zipper 2 8589934766 +luke allen 1 4294967533 +luke allen 2 8589934836 +luke allen 3 12884902346 +luke allen 4 17179869863 +luke allen 5 21474837208 +luke brown 1 4294967473 +luke davidson 1 4294967550 +luke davidson 2 8589934904 +luke ellison 1 4294967322 +luke ellison 2 8589934675 +luke ellison 3 12884902103 +luke falkner 1 4294967359 +luke falkner 2 8589934782 +luke garcia 1 4294967304 +luke garcia 2 8589934683 +luke ichabod 1 4294967324 +luke ichabod 2 8589934766 +luke johnson 1 4294967527 +luke johnson 2 8589934987 +luke johnson 3 12884902342 +luke laertes 1 4294967505 +luke laertes 2 8589935011 +luke laertes 3 12884902497 +luke laertes 4 17179869806 +luke laertes 5 21474837193 +luke miller 1 4294967497 +luke ovid 1 4294967492 +luke ovid 2 8589934901 +luke polk 1 4294967545 +luke polk 2 8589934873 +luke quirinius 1 4294967320 +luke robinson 1 4294967299 +luke robinson 2 8589934606 +luke thompson 1 4294967521 +luke underhill 1 4294967393 +luke underhill 2 8589934803 +luke underhill 3 12884902122 +luke van buren 1 4294967424 +luke white 1 4294967505 +luke xylophone 1 4294967382 +luke zipper 1 4294967353 +mike allen 1 4294967466 +mike brown 1 4294967369 +mike carson 1 4294967477 +mike carson 2 8589934934 +mike carson 3 12884902482 +mike davidson 1 4294967501 +mike davidson 2 8589934965 +mike ellison 1 4294967353 +mike ellison 2 8589934747 +mike ellison 3 12884902282 +mike ellison 4 17179869806 +mike ellison 5 21474837309 +mike falkner 1 4294967301 +mike garcia 1 4294967428 +mike garcia 2 8589934826 +mike garcia 3 12884902289 +mike hernandez 1 4294967316 +mike hernandez 2 8589934800 +mike ichabod 1 4294967494 +mike king 1 4294967323 +mike king 2 8589934848 +mike king 3 12884902248 +mike king 4 17179869595 +mike king 5 21474837046 +mike king 6 25769804478 +mike miller 1 4294967449 +mike nixon 1 4294967527 +mike nixon 2 8589935004 +mike polk 1 4294967389 +mike polk 2 8589934848 +mike polk 3 12884902351 +mike quirinius 1 4294967422 +mike steinbeck 1 4294967519 +mike steinbeck 2 8589934827 +mike steinbeck 3 12884902316 +mike steinbeck 4 17179869850 +mike van buren 1 4294967544 +mike van buren 2 8589935061 +mike white 1 4294967336 +mike white 2 8589934882 +mike white 3 12884902374 +mike white 4 17179869843 +mike young 1 4294967453 +mike young 2 8589934804 +mike young 3 12884902198 +mike zipper 1 4294967402 +mike zipper 2 8589934727 +mike zipper 3 12884902228 +nick allen 1 4294967507 +nick allen 2 8589934807 +nick brown 1 4294967334 +nick davidson 1 4294967357 +nick ellison 1 4294967397 +nick ellison 2 8589934699 +nick falkner 1 4294967480 +nick falkner 2 8589934923 +nick garcia 1 4294967384 +nick garcia 2 8589934797 +nick garcia 3 12884902319 +nick ichabod 1 4294967388 +nick ichabod 2 8589934758 +nick ichabod 3 12884902225 +nick johnson 1 4294967398 +nick johnson 2 8589934809 +nick laertes 1 4294967389 +nick miller 1 4294967550 +nick nixon 1 4294967482 +nick ovid 1 4294967488 +nick polk 1 4294967551 +nick quirinius 1 4294967316 +nick quirinius 2 8589934612 +nick robinson 1 4294967409 +nick robinson 2 8589934731 +nick steinbeck 1 4294967355 +nick thompson 1 4294967401 +nick underhill 1 4294967527 +nick van buren 1 4294967303 +nick xylophone 1 4294967460 +nick young 1 4294967405 +nick young 2 8589934917 +nick zipper 1 4294967430 +nick zipper 2 8589934796 +oscar allen 1 4294967500 +oscar brown 1 4294967331 +oscar carson 1 4294967460 +oscar carson 2 8589934904 +oscar carson 3 12884902286 +oscar carson 4 17179869599 +oscar carson 5 21474836960 +oscar davidson 1 4294967482 +oscar ellison 1 8589934740 +oscar ellison 1 8589934740 +oscar falkner 1 4294967526 +oscar garcia 1 4294967301 +oscar hernandez 1 4294967343 +oscar hernandez 2 8589934843 +oscar ichabod 1 4294967513 +oscar ichabod 2 8589934837 +oscar ichabod 3 12884902165 +oscar ichabod 4 17179869569 +oscar johnson 1 4294967418 +oscar johnson 2 8589934763 +oscar king 1 4294967465 +oscar king 2 8589934936 +oscar king 3 12884902469 +oscar laertes 1 4294967425 +oscar laertes 2 8589934876 +oscar laertes 3 12884902426 +oscar laertes 4 17179869786 +oscar nixon 1 4294967532 +oscar ovid 1 4294967508 +oscar ovid 2 8589934910 +oscar ovid 3 12884902418 +oscar polk 1 4294967325 +oscar polk 2 8589934713 +oscar quirinius 1 4294967416 +oscar quirinius 2 8589934932 +oscar quirinius 3 12884902390 +oscar quirinius 4 17179869763 +oscar robinson 1 4294967355 +oscar robinson 2 8589934681 +oscar robinson 3 12884902031 +oscar robinson 4 17179869383 +oscar steinbeck 1 4294967548 +oscar thompson 1 4294967453 +oscar thompson 2 8589934824 +oscar thompson 3 12884902197 +oscar thompson 4 17179869496 +oscar underhill 1 4294967374 +oscar van buren 1 4294967520 +oscar van buren 2 8589934990 +oscar van buren 3 12884902490 +oscar white 1 4294967454 +oscar white 2 8589934761 +oscar white 3 12884902163 +oscar white 4 17179869512 +oscar xylophone 1 4294967400 +oscar xylophone 2 8589934806 +oscar xylophone 3 12884902124 +oscar zipper 1 4294967449 +oscar zipper 2 8589934969 +oscar zipper 3 12884902458 +priscilla brown 1 4294967369 +priscilla brown 2 8589934897 +priscilla brown 3 12884902360 +priscilla carson 1 4294967489 +priscilla carson 2 8589934838 +priscilla carson 3 12884902270 +priscilla ichabod 1 4294967379 +priscilla ichabod 2 8589934926 +priscilla johnson 1 4294967535 +priscilla johnson 2 8589935003 +priscilla johnson 3 12884902308 +priscilla johnson 4 17179869707 +priscilla johnson 5 21474837167 +priscilla king 1 4294967385 +priscilla nixon 1 4294967388 +priscilla nixon 2 8589934849 +priscilla ovid 1 4294967528 +priscilla ovid 2 8589935035 +priscilla polk 1 4294967434 +priscilla quirinius 1 4294967347 +priscilla thompson 1 4294967497 +priscilla underhill 1 4294967520 +priscilla underhill 2 8589934853 +priscilla van buren 1 4294967318 +priscilla van buren 2 8589934809 +priscilla van buren 3 12884902351 +priscilla white 1 4294967419 +priscilla xylophone 1 4294967503 +priscilla xylophone 2 8589934956 +priscilla xylophone 3 12884902406 +priscilla young 1 4294967401 +priscilla young 2 8589934931 +priscilla zipper 1 4294967516 +priscilla zipper 2 8589934950 +quinn allen 1 4294967339 +quinn allen 2 8589934881 +quinn brown 1 4294967335 +quinn brown 2 8589934651 +quinn brown 3 12884902099 +quinn davidson 1 4294967478 +quinn davidson 2 8589934849 +quinn davidson 3 12884902238 +quinn davidson 4 17179869565 +quinn ellison 1 4294967392 +quinn ellison 2 8589934907 +quinn garcia 1 4294967344 +quinn garcia 2 8589934882 +quinn garcia 3 12884902395 +quinn garcia 4 17179869868 +quinn ichabod 1 4294967405 +quinn king 1 4294967538 +quinn king 2 8589934996 +quinn laertes 1 4294967533 +quinn laertes 2 8589934977 +quinn laertes 3 12884902524 +quinn nixon 1 4294967432 +quinn ovid 1 4294967340 +quinn quirinius 1 4294967347 +quinn robinson 1 4294967365 +quinn steinbeck 1 4294967358 +quinn steinbeck 2 8589934810 +quinn thompson 1 4294967488 +quinn thompson 2 8589934888 +quinn underhill 1 4294967307 +quinn underhill 2 8589934744 +quinn underhill 3 12884902278 +quinn van buren 1 4294967362 +quinn young 1 4294967392 +quinn zipper 1 4294967521 +quinn zipper 2 8589934944 +rachel allen 1 4294967334 +rachel allen 2 8589934713 +rachel brown 1 4294967451 +rachel brown 2 8589934886 +rachel brown 3 12884902325 +rachel brown 4 17179869632 +rachel brown 5 21474836938 +rachel carson 1 4294967461 +rachel carson 2 8589934777 +rachel davidson 1 4294967387 +rachel ellison 1 4294967423 +rachel falkner 1 4294967544 +rachel falkner 2 8589934892 +rachel falkner 3 12884902350 +rachel falkner 4 17179869809 +rachel johnson 1 4294967541 +rachel king 1 4294967442 +rachel king 2 8589934771 +rachel laertes 1 4294967446 +rachel laertes 2 8589934804 +rachel ovid 1 4294967481 +rachel ovid 2 8589934832 +rachel polk 1 4294967335 +rachel quirinius 1 4294967297 +rachel robinson 1 4294967344 +rachel robinson 2 8589934807 +rachel robinson 3 12884902135 +rachel thompson 1 4294967518 +rachel thompson 2 8589934881 +rachel thompson 3 12884902306 +rachel underhill 1 4294967382 +rachel white 1 4294967457 +rachel white 2 8589934793 +rachel young 1 4294967391 +rachel zipper 1 4294967434 +rachel zipper 2 8589934813 +sarah carson 1 4294967503 +sarah carson 2 8589934822 +sarah carson 3 12884902167 +sarah ellison 1 4294967542 +sarah falkner 1 4294967525 +sarah falkner 2 8589934974 +sarah garcia 1 4294967391 +sarah garcia 2 8589934849 +sarah garcia 3 12884902247 +sarah ichabod 1 4294967370 +sarah ichabod 2 8589934909 +sarah johnson 1 4294967433 +sarah johnson 2 8589934926 +sarah johnson 3 12884902235 +sarah johnson 4 17179869559 +sarah king 1 4294967496 +sarah king 2 8589935039 +sarah miller 1 4294967458 +sarah ovid 1 4294967350 +sarah robinson 1 4294967419 +sarah robinson 2 8589934917 +sarah steinbeck 1 4294967456 +sarah white 1 4294967514 +sarah white 2 8589934882 +sarah xylophone 1 4294967355 +sarah young 1 4294967442 +sarah zipper 1 4294967432 +tom brown 1 4294967432 +tom brown 2 8589934950 +tom carson 1 4294967388 +tom carson 2 8589934693 +tom carson 3 12884902227 +tom davidson 1 4294967507 +tom ellison 1 4294967487 +tom ellison 2 8589934844 +tom ellison 3 12884902188 +tom falkner 1 4294967382 +tom falkner 2 8589934837 +tom hernandez 1 8589934733 +tom hernandez 1 8589934733 +tom ichabod 1 4294967445 +tom johnson 1 4294967492 +tom johnson 2 8589934923 +tom king 1 4294967331 +tom laertes 1 4294967431 +tom laertes 2 8589934744 +tom miller 1 4294967366 +tom miller 2 8589934723 +tom miller 3 12884902078 +tom nixon 1 4294967506 +tom ovid 1 4294967512 +tom polk 1 4294967329 +tom polk 2 8589934869 +tom quirinius 1 4294967507 +tom quirinius 2 8589934823 +tom robinson 1 4294967457 +tom robinson 2 8589935008 +tom robinson 3 12884902462 +tom robinson 4 17179869770 +tom steinbeck 1 4294967447 +tom van buren 1 4294967374 +tom van buren 2 8589934703 +tom van buren 3 12884902195 +tom white 1 4294967413 +tom young 1 4294967539 +tom young 2 8589935074 +tom zipper 1 4294967526 +ulysses brown 1 4294967537 +ulysses carson 1 4294967323 +ulysses carson 2 8589934815 +ulysses carson 3 12884902127 +ulysses carson 4 17179869485 +ulysses davidson 1 4294967467 +ulysses ellison 1 4294967442 +ulysses garcia 1 4294967470 +ulysses hernandez 1 4294967449 +ulysses hernandez 2 8589934995 +ulysses hernandez 3 12884902393 +ulysses ichabod 1 4294967353 +ulysses ichabod 2 8589934728 +ulysses johnson 1 4294967432 +ulysses king 1 4294967537 +ulysses laertes 1 4294967391 +ulysses laertes 2 8589934938 +ulysses laertes 3 12884902431 +ulysses miller 1 4294967373 +ulysses miller 2 8589934808 +ulysses nixon 1 4294967296 +ulysses ovid 1 4294967394 +ulysses polk 1 4294967509 +ulysses polk 2 8589934960 +ulysses polk 3 12884902440 +ulysses polk 4 17179869745 +ulysses quirinius 1 4294967449 +ulysses robinson 1 4294967531 +ulysses steinbeck 1 4294967303 +ulysses steinbeck 2 8589934788 +ulysses thompson 1 4294967389 +ulysses underhill 1 4294967544 +ulysses underhill 2 8589934949 +ulysses underhill 3 12884902275 +ulysses underhill 4 17179869726 +ulysses underhill 5 21474837190 +ulysses underhill 6 25769804570 +ulysses underhill 7 30064771927 +ulysses van buren 1 4294967439 +ulysses white 1 4294967429 +ulysses white 2 8589934878 +ulysses xylophone 1 4294967524 +ulysses xylophone 2 8589935025 +ulysses xylophone 3 12884902473 +ulysses young 1 4294967427 +ulysses young 2 8589934763 +ulysses young 3 12884902154 +victor allen 1 4294967450 +victor allen 2 8589934776 +victor brown 1 4294967521 +victor brown 2 8589934864 +victor brown 3 12884902170 +victor brown 4 17179869625 +victor davidson 1 4294967419 +victor davidson 2 8589934720 +victor davidson 3 12884902156 +victor ellison 1 4294967362 +victor ellison 2 8589934831 +victor hernandez 1 4294967428 +victor hernandez 2 8589934733 +victor hernandez 3 12884902062 +victor hernandez 4 17179869402 +victor hernandez 5 21474836874 +victor johnson 1 4294967496 +victor johnson 2 8589934824 +victor johnson 3 12884902246 +victor king 1 4294967401 +victor king 2 8589934884 +victor laertes 1 4294967407 +victor laertes 2 8589934862 +victor miller 1 4294967410 +victor nixon 1 4294967424 +victor nixon 2 8589934803 +victor ovid 1 4294967355 +victor polk 1 4294967333 +victor quirinius 1 4294967520 +victor quirinius 2 8589934846 +victor robinson 1 4294967440 +victor robinson 2 8589934930 +victor steinbeck 1 4294967390 +victor steinbeck 2 8589934707 +victor steinbeck 3 12884902037 +victor thompson 1 4294967319 +victor van buren 1 4294967365 +victor van buren 2 8589934906 +victor white 1 4294967403 +victor white 2 8589934862 +victor xylophone 1 4294967331 +victor xylophone 2 8589934864 +victor xylophone 3 12884902262 +victor xylophone 4 17179869633 +victor xylophone 5 21474837062 +victor young 1 4294967337 +victor zipper 1 4294967428 +wendy allen 1 4294967473 +wendy allen 2 8589934989 +wendy allen 3 12884902367 +wendy brown 1 4294967337 +wendy brown 2 8589934817 +wendy ellison 1 4294967475 +wendy ellison 2 8589934989 +wendy falkner 1 4294967313 +wendy falkner 2 8589934810 +wendy falkner 3 12884902236 +wendy garcia 1 4294967394 +wendy garcia 2 8589934775 +wendy garcia 3 12884902088 +wendy garcia 4 17179869400 +wendy hernandez 1 4294967299 +wendy ichabod 1 4294967516 +wendy king 1 4294967420 +wendy king 2 8589934811 +wendy king 3 12884902252 +wendy laertes 1 4294967519 +wendy laertes 2 8589934939 +wendy laertes 3 12884902315 +wendy miller 1 4294967478 +wendy miller 2 8589934957 +wendy nixon 1 4294967407 +wendy nixon 2 8589934901 +wendy ovid 1 4294967464 +wendy ovid 2 8589934894 +wendy polk 1 4294967434 +wendy polk 2 8589934824 +wendy quirinius 1 4294967334 +wendy quirinius 2 8589934782 +wendy robinson 1 4294967302 +wendy robinson 2 8589934613 +wendy robinson 3 12884901977 +wendy steinbeck 1 4294967444 +wendy thompson 1 4294967301 +wendy thompson 2 8589934621 +wendy underhill 1 4294967540 +wendy underhill 2 8589934993 +wendy underhill 3 12884902410 +wendy van buren 1 4294967488 +wendy van buren 2 8589934835 +wendy white 1 4294967490 +wendy xylophone 1 4294967488 +wendy xylophone 2 8589934939 +wendy young 1 4294967395 +wendy young 2 8589934708 +xavier allen 1 4294967304 +xavier allen 2 8589934743 +xavier allen 3 12884902129 +xavier brown 1 4294967546 +xavier brown 2 8589935074 +xavier brown 3 12884902532 +xavier carson 1 4294967547 +xavier carson 2 8589934862 +xavier davidson 1 4294967361 +xavier davidson 2 8589934760 +xavier davidson 3 12884902204 +xavier ellison 1 4294967441 +xavier ellison 2 8589934914 +xavier garcia 1 4294967465 +xavier hernandez 1 4294967383 +xavier hernandez 2 8589934743 +xavier hernandez 3 12884902274 +xavier ichabod 1 4294967511 +xavier ichabod 2 8589934950 +xavier johnson 1 4294967507 +xavier johnson 2 8589934898 +xavier king 1 4294967456 +xavier king 2 8589934758 +xavier laertes 1 4294967450 +xavier ovid 1 4294967403 +xavier polk 1 4294967506 +xavier polk 2 8589934925 +xavier polk 3 12884902406 +xavier polk 4 17179869906 +xavier quirinius 1 4294967383 +xavier quirinius 2 8589934748 +xavier quirinius 3 12884902060 +xavier quirinius 4 17179869562 +xavier thompson 1 4294967444 +xavier underhill 1 4294967332 +xavier white 1 4294967473 +xavier white 2 8589934952 +xavier xylophone 1 4294967499 +xavier zipper 1 4294967547 +yuri allen 1 4294967528 +yuri allen 2 8589935079 +yuri brown 1 4294967433 +yuri brown 2 8589934960 +yuri carson 1 4294967317 +yuri carson 2 8589934851 +yuri ellison 1 4294967299 +yuri ellison 2 8589934697 +yuri falkner 1 4294967368 +yuri falkner 2 8589934891 +yuri garcia 1 4294967362 +yuri hernandez 1 4294967367 +yuri johnson 1 4294967421 +yuri johnson 2 8589934877 +yuri johnson 3 12884902361 +yuri king 1 4294967376 +yuri laertes 1 4294967402 +yuri laertes 2 8589934924 +yuri nixon 1 4294967400 +yuri nixon 2 8589934706 +yuri polk 1 4294967391 +yuri polk 2 8589934861 +yuri polk 3 12884902167 +yuri quirinius 1 4294967398 +yuri quirinius 2 8589934768 +yuri quirinius 3 12884902081 +yuri steinbeck 1 4294967535 +yuri steinbeck 2 8589934873 +yuri thompson 1 4294967447 +yuri underhill 1 4294967499 +yuri underhill 2 8589934900 +yuri white 1 4294967341 +yuri xylophone 1 4294967420 +zach allen 1 4294967507 +zach brown 1 4294967316 +zach brown 2 8589934728 +zach brown 3 12884902099 +zach brown 4 17179869452 +zach brown 5 21474836769 +zach carson 1 4294967463 +zach ellison 1 4294967471 +zach falkner 1 4294967362 +zach falkner 2 8589934717 +zach garcia 1 4294967481 +zach garcia 2 8589934854 +zach garcia 3 12884902240 +zach garcia 4 17179869723 +zach ichabod 1 4294967539 +zach ichabod 2 8589934912 +zach king 1 4294967424 +zach king 2 8589934956 +zach king 3 12884902458 +zach miller 1 4294967442 +zach miller 2 8589934772 +zach miller 3 12884902163 +zach ovid 1 4294967412 +zach ovid 2 8589934775 +zach ovid 3 12884902244 +zach ovid 4 17179869574 +zach quirinius 1 4294967299 +zach robinson 1 4294967325 +zach steinbeck 1 4294967469 +zach steinbeck 2 8589934834 +zach thompson 1 4294967405 +zach thompson 2 8589934730 +zach underhill 1 4294967496 +zach white 1 4294967501 +zach xylophone 1 4294967452 +zach xylophone 2 8589934755 +zach young 1 4294967297 +zach zipper 1 4294967497 +zach zipper 2 8589934855 +zach zipper 3 12884902222 diff --git a/sql/hive/src/test/resources/golden/windowing_multipartitioning.q (deterministic) 3-0-455e41d9949a2d22bab634fd8e42f2b1 b/sql/hive/src/test/resources/golden/windowing_multipartitioning.q (deterministic) 3-0-455e41d9949a2d22bab634fd8e42f2b1 new file mode 100644 index 0000000000000..f47923618a1aa --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_multipartitioning.q (deterministic) 3-0-455e41d9949a2d22bab634fd8e42f2b1 @@ -0,0 +1 @@ +bob steinbeck 65637 9.699999809265137 diff --git a/sql/hive/src/test/resources/golden/windowing_multipartitioning.q (deterministic) 4-0-cfad06ae8eba6b047d32a6a61dd59392 b/sql/hive/src/test/resources/golden/windowing_multipartitioning.q (deterministic) 4-0-cfad06ae8eba6b047d32a6a61dd59392 new file mode 100644 index 0000000000000..f41eaa259cec0 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_multipartitioning.q (deterministic) 4-0-cfad06ae8eba6b047d32a6a61dd59392 @@ -0,0 +1 @@ +bob steinbeck 1 1 diff --git a/sql/hive/src/test/resources/golden/windowing_multipartitioning.q (deterministic) 5-0-d7ca7a61377cef3a9f721a28afdae012 b/sql/hive/src/test/resources/golden/windowing_multipartitioning.q (deterministic) 5-0-d7ca7a61377cef3a9f721a28afdae012 new file mode 100644 index 0000000000000..5308b2eb457e7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_multipartitioning.q (deterministic) 5-0-d7ca7a61377cef3a9f721a28afdae012 @@ -0,0 +1 @@ +bob steinbeck 9.699999809265137 1 diff --git a/sql/hive/src/test/resources/golden/windowing_multipartitioning.q (deterministic) 6-0-287bcc7679822bc7b684532b267bf11f b/sql/hive/src/test/resources/golden/windowing_multipartitioning.q (deterministic) 6-0-287bcc7679822bc7b684532b267bf11f new file mode 100644 index 0000000000000..f41eaa259cec0 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_multipartitioning.q (deterministic) 6-0-287bcc7679822bc7b684532b267bf11f @@ -0,0 +1 @@ +bob steinbeck 1 1 diff --git a/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-0-36217f6074daaacddb9fcb50a3f4fb5b b/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-0-36217f6074daaacddb9fcb50a3f4fb5b new file mode 100644 index 0000000000000..8150409e62d33 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-0-36217f6074daaacddb9fcb50a3f4fb5b @@ -0,0 +1,1049 @@ + 1 + 1 + 1 +alice allen 2 +alice allen 1 +alice allen 1 +alice brown 1 +alice carson 1 +alice davidson 1 +alice falkner 1 +alice garcia 1 +alice hernandez 2 +alice hernandez 1 +alice johnson 2 +alice king 1 +alice king 1 +alice king 1 +alice laertes 1 +alice laertes 1 +alice miller 1 +alice nixon 1 +alice nixon 1 +alice nixon 1 +alice ovid 1 +alice polk 3 +alice quirinius 1 +alice quirinius 1 +alice robinson 1 +alice robinson 1 +alice steinbeck 1 +alice steinbeck 1 +alice steinbeck 1 +alice underhill 2 +alice van buren 1 +alice xylophone 1 +alice xylophone 1 +alice xylophone 1 +alice zipper 1 +alice zipper 1 +alice zipper 1 +bob brown 1 +bob brown 1 +bob brown 1 +bob carson 1 +bob davidson 1 +bob davidson 1 +bob davidson 1 +bob ellison 2 +bob ellison 1 +bob ellison 1 +bob ellison 1 +bob falkner 1 +bob garcia 1 +bob garcia 1 +bob garcia 1 +bob garcia 1 +bob garcia 1 +bob hernandez 1 +bob ichabod 1 +bob king 2 +bob king 1 +bob king 1 +bob laertes 2 +bob laertes 1 +bob miller 1 +bob ovid 1 +bob ovid 1 +bob ovid 1 +bob ovid 1 +bob polk 1 +bob quirinius 1 +bob steinbeck 1 +bob van buren 1 +bob white 1 +bob white 1 +bob xylophone 1 +bob xylophone 1 +bob young 1 +bob zipper 2 +bob zipper 1 +bob zipper 1 +calvin allen 1 +calvin brown 2 +calvin brown 1 +calvin brown 1 +calvin carson 2 +calvin davidson 2 +calvin davidson 1 +calvin ellison 1 +calvin falkner 1 +calvin falkner 1 +calvin falkner 1 +calvin falkner 1 +calvin falkner 1 +calvin falkner 1 +calvin garcia 1 +calvin hernandez 2 +calvin johnson 1 +calvin laertes 1 +calvin laertes 1 +calvin nixon 3 +calvin nixon 1 +calvin nixon 1 +calvin ovid 1 +calvin ovid 1 +calvin ovid 1 +calvin ovid 1 +calvin polk 1 +calvin quirinius 1 +calvin quirinius 1 +calvin robinson 2 +calvin steinbeck 1 +calvin steinbeck 1 +calvin steinbeck 1 +calvin thompson 1 +calvin thompson 1 +calvin underhill 1 +calvin van buren 1 +calvin van buren 1 +calvin white 1 +calvin white 1 +calvin xylophone 2 +calvin xylophone 1 +calvin xylophone 1 +calvin young 1 +calvin young 1 +calvin zipper 1 +calvin zipper 1 +david allen 1 +david allen 1 +david brown 1 +david brown 1 +david davidson 1 +david davidson 1 +david davidson 1 +david davidson 1 +david ellison 1 +david ellison 1 +david ellison 1 +david hernandez 1 +david ichabod 1 +david ichabod 1 +david laertes 1 +david nixon 1 +david ovid 1 +david ovid 1 +david quirinius 1 +david quirinius 1 +david quirinius 1 +david robinson 1 +david robinson 1 +david thompson 3 +david underhill 1 +david underhill 1 +david underhill 1 +david van buren 1 +david van buren 1 +david white 1 +david xylophone 1 +david xylophone 1 +david xylophone 1 +david young 1 +david young 1 +ethan allen 1 +ethan brown 2 +ethan brown 2 +ethan brown 1 +ethan brown 1 +ethan brown 1 +ethan brown 1 +ethan carson 1 +ethan ellison 1 +ethan ellison 1 +ethan falkner 3 +ethan falkner 1 +ethan garcia 1 +ethan hernandez 1 +ethan johnson 1 +ethan king 1 +ethan laertes 1 +ethan laertes 1 +ethan laertes 1 +ethan laertes 1 +ethan laertes 1 +ethan laertes 1 +ethan laertes 1 +ethan miller 2 +ethan nixon 1 +ethan ovid 1 +ethan polk 2 +ethan polk 1 +ethan polk 1 +ethan polk 1 +ethan quirinius 1 +ethan quirinius 1 +ethan quirinius 1 +ethan robinson 1 +ethan robinson 1 +ethan underhill 1 +ethan van buren 2 +ethan white 1 +ethan white 1 +ethan xylophone 1 +ethan zipper 1 +ethan zipper 1 +fred davidson 1 +fred davidson 1 +fred davidson 1 +fred ellison 1 +fred ellison 1 +fred ellison 1 +fred falkner 2 +fred falkner 1 +fred falkner 1 +fred hernandez 1 +fred ichabod 1 +fred ichabod 1 +fred johnson 1 +fred king 2 +fred king 1 +fred laertes 1 +fred miller 1 +fred nixon 2 +fred nixon 1 +fred nixon 1 +fred nixon 1 +fred polk 1 +fred polk 1 +fred polk 1 +fred polk 1 +fred quirinius 1 +fred quirinius 1 +fred robinson 1 +fred steinbeck 1 +fred steinbeck 1 +fred steinbeck 1 +fred underhill 1 +fred van buren 1 +fred van buren 1 +fred van buren 1 +fred van buren 1 +fred white 1 +fred young 3 +fred young 1 +fred zipper 1 +gabriella allen 1 +gabriella allen 1 +gabriella brown 2 +gabriella brown 1 +gabriella carson 1 +gabriella davidson 1 +gabriella ellison 1 +gabriella ellison 1 +gabriella falkner 1 +gabriella falkner 1 +gabriella falkner 1 +gabriella garcia 1 +gabriella hernandez 1 +gabriella hernandez 1 +gabriella ichabod 1 +gabriella ichabod 1 +gabriella ichabod 1 +gabriella ichabod 1 +gabriella ichabod 1 +gabriella king 1 +gabriella king 1 +gabriella laertes 1 +gabriella miller 1 +gabriella ovid 1 +gabriella ovid 1 +gabriella polk 1 +gabriella polk 1 +gabriella steinbeck 1 +gabriella steinbeck 1 +gabriella thompson 1 +gabriella thompson 1 +gabriella thompson 1 +gabriella van buren 2 +gabriella van buren 1 +gabriella white 1 +gabriella young 1 +gabriella young 1 +gabriella zipper 1 +gabriella zipper 1 +holly allen 1 +holly brown 1 +holly brown 1 +holly falkner 1 +holly hernandez 1 +holly hernandez 1 +holly hernandez 1 +holly hernandez 1 +holly ichabod 1 +holly ichabod 1 +holly ichabod 1 +holly johnson 2 +holly johnson 1 +holly johnson 1 +holly king 1 +holly king 1 +holly laertes 1 +holly miller 1 +holly nixon 1 +holly nixon 1 +holly polk 2 +holly polk 1 +holly robinson 2 +holly thompson 1 +holly thompson 1 +holly thompson 1 +holly underhill 1 +holly underhill 1 +holly underhill 1 +holly underhill 1 +holly van buren 1 +holly white 1 +holly white 1 +holly xylophone 1 +holly young 1 +holly young 1 +holly zipper 1 +holly zipper 1 +irene allen 1 +irene brown 1 +irene brown 1 +irene brown 1 +irene carson 1 +irene ellison 1 +irene ellison 1 +irene falkner 1 +irene falkner 1 +irene garcia 1 +irene garcia 1 +irene garcia 1 +irene ichabod 1 +irene ichabod 1 +irene johnson 2 +irene laertes 1 +irene laertes 1 +irene laertes 1 +irene miller 2 +irene nixon 1 +irene nixon 1 +irene nixon 1 +irene ovid 1 +irene ovid 1 +irene ovid 1 +irene polk 1 +irene polk 1 +irene polk 1 +irene polk 1 +irene polk 1 +irene quirinius 1 +irene quirinius 1 +irene quirinius 1 +irene robinson 1 +irene steinbeck 1 +irene thompson 2 +irene underhill 2 +irene underhill 1 +irene van buren 1 +irene van buren 1 +irene xylophone 1 +jessica brown 2 +jessica carson 1 +jessica carson 1 +jessica carson 1 +jessica davidson 2 +jessica davidson 1 +jessica davidson 1 +jessica davidson 1 +jessica ellison 2 +jessica ellison 1 +jessica falkner 1 +jessica garcia 1 +jessica garcia 1 +jessica ichabod 1 +jessica johnson 1 +jessica johnson 1 +jessica miller 1 +jessica nixon 1 +jessica nixon 1 +jessica ovid 2 +jessica ovid 1 +jessica polk 1 +jessica quirinius 1 +jessica quirinius 1 +jessica quirinius 1 +jessica quirinius 1 +jessica robinson 1 +jessica thompson 1 +jessica thompson 1 +jessica underhill 1 +jessica underhill 1 +jessica underhill 1 +jessica van buren 1 +jessica white 2 +jessica white 1 +jessica white 1 +jessica white 1 +jessica white 1 +jessica xylophone 1 +jessica young 1 +jessica young 1 +jessica zipper 1 +jessica zipper 1 +jessica zipper 1 +katie allen 1 +katie brown 1 +katie davidson 1 +katie ellison 1 +katie ellison 1 +katie falkner 1 +katie garcia 1 +katie garcia 1 +katie hernandez 1 +katie ichabod 2 +katie ichabod 1 +katie ichabod 1 +katie king 1 +katie king 1 +katie king 1 +katie miller 1 +katie miller 1 +katie nixon 1 +katie ovid 1 +katie polk 1 +katie polk 1 +katie robinson 1 +katie van buren 2 +katie van buren 1 +katie white 1 +katie white 1 +katie xylophone 1 +katie young 1 +katie young 1 +katie young 1 +katie zipper 1 +katie zipper 1 +luke allen 2 +luke allen 1 +luke allen 1 +luke allen 1 +luke allen 1 +luke brown 2 +luke davidson 1 +luke davidson 1 +luke ellison 1 +luke ellison 1 +luke ellison 1 +luke falkner 2 +luke falkner 1 +luke garcia 1 +luke garcia 1 +luke ichabod 1 +luke ichabod 1 +luke johnson 1 +luke johnson 1 +luke johnson 1 +luke laertes 1 +luke laertes 1 +luke laertes 1 +luke laertes 1 +luke laertes 1 +luke miller 2 +luke ovid 2 +luke ovid 1 +luke polk 1 +luke polk 1 +luke quirinius 1 +luke robinson 1 +luke robinson 1 +luke thompson 1 +luke underhill 1 +luke underhill 1 +luke underhill 1 +luke van buren 2 +luke white 1 +luke xylophone 1 +luke zipper 1 +mike allen 2 +mike brown 1 +mike carson 1 +mike carson 1 +mike carson 1 +mike davidson 1 +mike davidson 1 +mike ellison 2 +mike ellison 1 +mike ellison 1 +mike ellison 1 +mike ellison 1 +mike falkner 1 +mike garcia 1 +mike garcia 1 +mike garcia 1 +mike hernandez 1 +mike hernandez 1 +mike ichabod 1 +mike king 2 +mike king 1 +mike king 1 +mike king 1 +mike king 1 +mike king 1 +mike miller 1 +mike nixon 2 +mike nixon 1 +mike polk 2 +mike polk 1 +mike polk 1 +mike quirinius 1 +mike steinbeck 1 +mike steinbeck 1 +mike steinbeck 1 +mike steinbeck 1 +mike van buren 2 +mike van buren 1 +mike white 1 +mike white 1 +mike white 1 +mike white 1 +mike young 1 +mike young 1 +mike young 1 +mike zipper 1 +mike zipper 1 +mike zipper 1 +nick allen 1 +nick allen 1 +nick brown 1 +nick davidson 1 +nick ellison 2 +nick ellison 1 +nick falkner 1 +nick falkner 1 +nick garcia 1 +nick garcia 1 +nick garcia 1 +nick ichabod 1 +nick ichabod 1 +nick ichabod 1 +nick johnson 1 +nick johnson 1 +nick laertes 1 +nick miller 1 +nick nixon 1 +nick ovid 1 +nick polk 2 +nick quirinius 2 +nick quirinius 1 +nick robinson 1 +nick robinson 1 +nick steinbeck 1 +nick thompson 2 +nick underhill 1 +nick van buren 1 +nick xylophone 1 +nick young 1 +nick young 1 +nick zipper 2 +nick zipper 1 +oscar allen 1 +oscar brown 1 +oscar carson 1 +oscar carson 1 +oscar carson 1 +oscar carson 1 +oscar carson 1 +oscar davidson 1 +oscar ellison 2 +oscar ellison 2 +oscar falkner 1 +oscar garcia 1 +oscar hernandez 1 +oscar hernandez 1 +oscar ichabod 2 +oscar ichabod 1 +oscar ichabod 1 +oscar ichabod 1 +oscar johnson 2 +oscar johnson 1 +oscar king 1 +oscar king 1 +oscar king 1 +oscar laertes 1 +oscar laertes 1 +oscar laertes 1 +oscar laertes 1 +oscar nixon 1 +oscar ovid 1 +oscar ovid 1 +oscar ovid 1 +oscar polk 1 +oscar polk 1 +oscar quirinius 2 +oscar quirinius 2 +oscar quirinius 1 +oscar quirinius 1 +oscar robinson 2 +oscar robinson 1 +oscar robinson 1 +oscar robinson 1 +oscar steinbeck 1 +oscar thompson 1 +oscar thompson 1 +oscar thompson 1 +oscar thompson 1 +oscar underhill 1 +oscar van buren 1 +oscar van buren 1 +oscar van buren 1 +oscar white 1 +oscar white 1 +oscar white 1 +oscar white 1 +oscar xylophone 2 +oscar xylophone 1 +oscar xylophone 1 +oscar zipper 2 +oscar zipper 1 +oscar zipper 1 +priscilla brown 2 +priscilla brown 1 +priscilla brown 1 +priscilla carson 2 +priscilla carson 1 +priscilla carson 1 +priscilla ichabod 2 +priscilla ichabod 1 +priscilla johnson 1 +priscilla johnson 1 +priscilla johnson 1 +priscilla johnson 1 +priscilla johnson 1 +priscilla king 1 +priscilla nixon 2 +priscilla nixon 2 +priscilla ovid 2 +priscilla ovid 1 +priscilla polk 1 +priscilla quirinius 1 +priscilla thompson 1 +priscilla underhill 1 +priscilla underhill 1 +priscilla van buren 2 +priscilla van buren 1 +priscilla van buren 1 +priscilla white 1 +priscilla xylophone 1 +priscilla xylophone 1 +priscilla xylophone 1 +priscilla young 1 +priscilla young 1 +priscilla zipper 1 +priscilla zipper 1 +quinn allen 1 +quinn allen 1 +quinn brown 1 +quinn brown 1 +quinn brown 1 +quinn davidson 2 +quinn davidson 1 +quinn davidson 1 +quinn davidson 1 +quinn ellison 1 +quinn ellison 1 +quinn garcia 1 +quinn garcia 1 +quinn garcia 1 +quinn garcia 1 +quinn ichabod 1 +quinn king 1 +quinn king 1 +quinn laertes 1 +quinn laertes 1 +quinn laertes 1 +quinn nixon 2 +quinn ovid 1 +quinn quirinius 1 +quinn robinson 2 +quinn steinbeck 1 +quinn steinbeck 1 +quinn thompson 1 +quinn thompson 1 +quinn underhill 2 +quinn underhill 1 +quinn underhill 1 +quinn van buren 1 +quinn young 2 +quinn zipper 1 +quinn zipper 1 +rachel allen 1 +rachel allen 1 +rachel brown 3 +rachel brown 1 +rachel brown 1 +rachel brown 1 +rachel brown 1 +rachel carson 2 +rachel carson 1 +rachel davidson 1 +rachel ellison 1 +rachel falkner 1 +rachel falkner 1 +rachel falkner 1 +rachel falkner 1 +rachel johnson 1 +rachel king 2 +rachel king 1 +rachel laertes 1 +rachel laertes 1 +rachel ovid 1 +rachel ovid 1 +rachel polk 1 +rachel quirinius 1 +rachel robinson 1 +rachel robinson 1 +rachel robinson 1 +rachel thompson 1 +rachel thompson 1 +rachel thompson 1 +rachel underhill 1 +rachel white 1 +rachel white 1 +rachel young 1 +rachel zipper 1 +rachel zipper 1 +sarah carson 1 +sarah carson 1 +sarah carson 1 +sarah ellison 1 +sarah falkner 1 +sarah falkner 1 +sarah garcia 1 +sarah garcia 1 +sarah garcia 1 +sarah ichabod 1 +sarah ichabod 1 +sarah johnson 1 +sarah johnson 1 +sarah johnson 1 +sarah johnson 1 +sarah king 1 +sarah king 1 +sarah miller 1 +sarah ovid 1 +sarah robinson 1 +sarah robinson 1 +sarah steinbeck 1 +sarah white 1 +sarah white 1 +sarah xylophone 1 +sarah young 1 +sarah zipper 1 +tom brown 1 +tom brown 1 +tom carson 1 +tom carson 1 +tom carson 1 +tom davidson 1 +tom ellison 1 +tom ellison 1 +tom ellison 1 +tom falkner 1 +tom falkner 1 +tom hernandez 1 +tom hernandez 1 +tom ichabod 1 +tom johnson 1 +tom johnson 1 +tom king 1 +tom laertes 2 +tom laertes 1 +tom miller 2 +tom miller 1 +tom miller 1 +tom nixon 2 +tom ovid 1 +tom polk 1 +tom polk 1 +tom quirinius 1 +tom quirinius 1 +tom robinson 1 +tom robinson 1 +tom robinson 1 +tom robinson 1 +tom steinbeck 3 +tom van buren 1 +tom van buren 1 +tom van buren 1 +tom white 2 +tom young 1 +tom young 1 +tom zipper 3 +ulysses brown 1 +ulysses carson 4 +ulysses carson 1 +ulysses carson 1 +ulysses carson 1 +ulysses davidson 2 +ulysses ellison 1 +ulysses garcia 1 +ulysses hernandez 1 +ulysses hernandez 1 +ulysses hernandez 1 +ulysses ichabod 1 +ulysses ichabod 1 +ulysses johnson 2 +ulysses king 1 +ulysses laertes 2 +ulysses laertes 1 +ulysses laertes 1 +ulysses miller 1 +ulysses miller 1 +ulysses nixon 1 +ulysses ovid 1 +ulysses polk 1 +ulysses polk 1 +ulysses polk 1 +ulysses polk 1 +ulysses quirinius 1 +ulysses robinson 1 +ulysses steinbeck 1 +ulysses steinbeck 1 +ulysses thompson 1 +ulysses underhill 2 +ulysses underhill 1 +ulysses underhill 1 +ulysses underhill 1 +ulysses underhill 1 +ulysses underhill 1 +ulysses underhill 1 +ulysses van buren 1 +ulysses white 1 +ulysses white 1 +ulysses xylophone 2 +ulysses xylophone 1 +ulysses xylophone 1 +ulysses young 2 +ulysses young 1 +ulysses young 1 +victor allen 1 +victor allen 1 +victor brown 1 +victor brown 1 +victor brown 1 +victor brown 1 +victor davidson 1 +victor davidson 1 +victor davidson 1 +victor ellison 2 +victor ellison 1 +victor hernandez 1 +victor hernandez 1 +victor hernandez 1 +victor hernandez 1 +victor hernandez 1 +victor johnson 2 +victor johnson 1 +victor johnson 1 +victor king 1 +victor king 1 +victor laertes 1 +victor laertes 1 +victor miller 1 +victor nixon 1 +victor nixon 1 +victor ovid 2 +victor polk 2 +victor quirinius 1 +victor quirinius 1 +victor robinson 2 +victor robinson 1 +victor steinbeck 2 +victor steinbeck 1 +victor steinbeck 1 +victor thompson 1 +victor van buren 1 +victor van buren 1 +victor white 2 +victor white 1 +victor xylophone 1 +victor xylophone 1 +victor xylophone 1 +victor xylophone 1 +victor xylophone 1 +victor young 1 +victor zipper 1 +wendy allen 1 +wendy allen 1 +wendy allen 1 +wendy brown 1 +wendy brown 1 +wendy ellison 1 +wendy ellison 1 +wendy falkner 2 +wendy falkner 1 +wendy falkner 1 +wendy garcia 2 +wendy garcia 1 +wendy garcia 1 +wendy garcia 1 +wendy hernandez 1 +wendy ichabod 1 +wendy king 1 +wendy king 1 +wendy king 1 +wendy laertes 1 +wendy laertes 1 +wendy laertes 1 +wendy miller 1 +wendy miller 1 +wendy nixon 1 +wendy nixon 1 +wendy ovid 1 +wendy ovid 1 +wendy polk 2 +wendy polk 1 +wendy quirinius 1 +wendy quirinius 1 +wendy robinson 2 +wendy robinson 1 +wendy robinson 1 +wendy steinbeck 1 +wendy thompson 2 +wendy thompson 1 +wendy underhill 2 +wendy underhill 1 +wendy underhill 1 +wendy van buren 1 +wendy van buren 1 +wendy white 1 +wendy xylophone 1 +wendy xylophone 1 +wendy young 1 +wendy young 1 +xavier allen 1 +xavier allen 1 +xavier allen 1 +xavier brown 1 +xavier brown 1 +xavier brown 1 +xavier carson 1 +xavier carson 1 +xavier davidson 1 +xavier davidson 1 +xavier davidson 1 +xavier ellison 1 +xavier ellison 1 +xavier garcia 1 +xavier hernandez 1 +xavier hernandez 1 +xavier hernandez 1 +xavier ichabod 1 +xavier ichabod 1 +xavier johnson 1 +xavier johnson 1 +xavier king 1 +xavier king 1 +xavier laertes 1 +xavier ovid 2 +xavier polk 1 +xavier polk 1 +xavier polk 1 +xavier polk 1 +xavier quirinius 2 +xavier quirinius 2 +xavier quirinius 1 +xavier quirinius 1 +xavier thompson 1 +xavier underhill 1 +xavier white 2 +xavier white 1 +xavier xylophone 1 +xavier zipper 1 +yuri allen 1 +yuri allen 1 +yuri brown 1 +yuri brown 1 +yuri carson 1 +yuri carson 1 +yuri ellison 1 +yuri ellison 1 +yuri falkner 1 +yuri falkner 1 +yuri garcia 1 +yuri hernandez 1 +yuri johnson 1 +yuri johnson 1 +yuri johnson 1 +yuri king 2 +yuri laertes 1 +yuri laertes 1 +yuri nixon 1 +yuri nixon 1 +yuri polk 1 +yuri polk 1 +yuri polk 1 +yuri quirinius 1 +yuri quirinius 1 +yuri quirinius 1 +yuri steinbeck 1 +yuri steinbeck 1 +yuri thompson 1 +yuri underhill 1 +yuri underhill 1 +yuri white 1 +yuri xylophone 1 +zach allen 1 +zach brown 2 +zach brown 1 +zach brown 1 +zach brown 1 +zach brown 1 +zach carson 2 +zach ellison 1 +zach falkner 1 +zach falkner 1 +zach garcia 2 +zach garcia 1 +zach garcia 1 +zach garcia 1 +zach ichabod 1 +zach ichabod 1 +zach king 2 +zach king 1 +zach king 1 +zach miller 1 +zach miller 1 +zach miller 1 +zach ovid 1 +zach ovid 1 +zach ovid 1 +zach ovid 1 +zach quirinius 1 +zach robinson 1 +zach steinbeck 1 +zach steinbeck 1 +zach thompson 2 +zach thompson 1 +zach underhill 1 +zach white 1 +zach xylophone 2 +zach xylophone 1 +zach young 1 +zach zipper 1 +zach zipper 1 +zach zipper 1 diff --git a/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-1-9ee79e711248dd6e0a6ce27e439e55f4 b/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-1-9ee79e711248dd6e0a6ce27e439e55f4 new file mode 100644 index 0000000000000..275772e1f643a --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-1-9ee79e711248dd6e0a6ce27e439e55f4 @@ -0,0 +1,1049 @@ +65791 calvin nixon +65791 katie garcia +65790 fred nixon +65790 victor polk +65790 yuri ellison +65789 NULL +65789 alice laertes +65789 gabriella king +65789 katie zipper +65789 oscar zipper +65789 quinn davidson +65789 wendy brown +65788 irene brown +65788 oscar zipper +65788 rachel king +65788 xavier thompson +65787 alice laertes +65787 david laertes +65787 katie ichabod +65787 ulysses king +65786 bob carson +65786 quinn king +65786 victor xylophone +65786 xavier allen +65786 xavier davidson +65785 sarah carson +65785 sarah johnson +65784 holly young +65784 jessica ellison +65784 jessica ovid +65784 jessica zipper +65784 quinn van buren +65783 david allen +65783 fred ellison +65783 irene nixon +65783 mike ichabod +65783 wendy miller +65783 zach garcia +65782 tom miller +65782 ulysses underhill +65782 victor nixon +65782 yuri white +65782 zach ovid +65781 ethan carson +65781 luke laertes +65781 quinn brown +65780 holly hernandez +65780 holly zipper +65780 wendy falkner +65779 gabriella ichabod +65779 irene brown +65779 irene underhill +65779 ulysses xylophone +65779 wendy hernandez +65779 yuri nixon +65779 zach ellison +65778 jessica davidson +65778 katie allen +65778 oscar van buren +65778 oscar white +65778 rachel johnson +65778 victor quirinius +65778 yuri polk +65778 yuri quirinius +65777 ethan garcia +65777 irene garcia +65777 katie polk +65777 rachel thompson +65776 NULL +65776 luke garcia +65776 luke quirinius +65776 priscilla ovid +65776 sarah king +65775 bob ellison +65775 calvin steinbeck +65775 ethan laertes +65775 luke robinson +65775 oscar polk +65774 calvin young +65774 irene quirinius +65774 katie brown +65774 oscar hernandez +65774 sarah ichabod +65774 zach king +65773 NULL +65773 calvin garcia +65773 irene polk +65773 jessica davidson +65773 nick ichabod +65773 nick zipper +65773 quinn ovid +65773 zach falkner +65772 oscar garcia +65771 ethan underhill +65771 yuri hernandez +65771 zach brown +65770 alice nixon +65770 gabriella ellison +65769 mike king +65769 nick steinbeck +65769 oscar nixon +65769 rachel zipper +65769 xavier quirinius +65769 zach ichabod +65768 jessica davidson +65767 tom robinson +65767 victor davidson +65767 xavier king +65766 fred davidson +65766 luke underhill +65766 ulysses carson +65766 wendy nixon +65766 xavier brown +65765 bob polk +65765 gabriella brown +65765 wendy allen +65764 alice hernandez +65764 alice robinson +65764 luke ellison +65763 calvin van buren +65763 ethan brown +65763 priscilla johnson +65763 tom ichabod +65763 tom miller +65762 david xylophone +65762 priscilla xylophone +65762 quinn allen +65762 sarah johnson +65762 tom carson +65761 david ovid +65760 NULL +65760 ethan king +65760 priscilla johnson +65760 wendy van buren +65759 alice xylophone +65759 ethan quirinius +65759 fred steinbeck +65759 sarah robinson +65759 xavier johnson +65758 alice van buren +65758 ethan white +65758 irene johnson +65758 jessica underhill +65758 rachel laertes +65757 bob quirinius +65757 nick quirinius +65757 tom van buren +65757 xavier carson +65757 zach brown +65756 gabriella falkner +65756 mike polk +65756 nick allen +65756 yuri xylophone +65755 alice johnson +65755 david ellison +65755 fred polk +65755 gabriella hernandez +65755 irene van buren +65755 jessica nixon +65755 mike ellison +65755 nick van buren +65755 zach miller +65754 luke johnson +65754 xavier quirinius +65753 rachel robinson +65753 yuri garcia +65752 bob miller +65752 oscar zipper +65751 calvin quirinius +65751 katie king +65751 mike allen +65751 mike quirinius +65751 mike white +65751 zach zipper +65750 mike white +65750 nick thompson +65750 oscar quirinius +65750 priscilla polk +65750 rachel brown +65749 david allen +65749 gabriella polk +65749 mike ellison +65749 sarah carson +65749 wendy thompson +65748 david davidson +65748 oscar laertes +65747 calvin falkner +65747 fred steinbeck +65747 priscilla zipper +65747 quinn underhill +65747 rachel falkner +65747 wendy falkner +65747 yuri falkner +65746 +65746 katie robinson +65746 luke garcia +65746 tom brown +65746 zach steinbeck +65745 oscar steinbeck +65745 oscar xylophone +65745 priscilla nixon +65745 victor laertes +65744 alice allen +65744 priscilla ichabod +65743 jessica carson +65743 oscar ichabod +65743 sarah falkner +65743 tom brown +65742 jessica brown +65742 jessica ellison +65742 wendy polk +65741 calvin thompson +65740 irene van buren +65740 mike zipper +65740 rachel quirinius +65739 gabriella van buren +65739 victor robinson +65739 wendy nixon +65738 sarah ichabod +65738 wendy robinson +65738 wendy young +65738 xavier king +65737 holly brown +65737 irene falkner +65737 jessica white +65737 quinn garcia +65737 wendy laertes +65736 fred young +65736 quinn laertes +65736 rachel allen +65736 victor ellison +65735 oscar laertes +65735 tom falkner +65735 ulysses thompson +65734 calvin falkner +65734 mike garcia +65733 gabriella van buren +65733 holly nixon +65733 luke underhill +65733 oscar allen +65733 xavier polk +65733 yuri ellison +65732 mike steinbeck +65732 tom carson +65732 tom ellison +65731 alice robinson +65731 priscilla carson +65731 tom johnson +65731 ulysses ovid +65730 bob king +65730 calvin polk +65730 gabriella ichabod +65730 rachel brown +65729 fred quirinius +65728 priscilla king +65728 victor brown +65727 ethan brown +65727 fred polk +65727 jessica white +65727 priscilla underhill +65727 sarah white +65726 mike garcia +65726 ulysses hernandez +65726 wendy quirinius +65725 zach thompson +65724 NULL +65724 alice king +65724 alice steinbeck +65724 mike ellison +65724 nick ovid +65724 priscilla zipper +65723 nick allen +65723 rachel white +65722 fred falkner +65722 jessica underhill +65722 luke ichabod +65721 ethan falkner +65721 jessica zipper +65721 luke laertes +65721 luke underhill +65721 mike carson +65721 oscar laertes +65721 ulysses ichabod +65720 calvin nixon +65720 calvin thompson +65720 gabriella young +65720 holly polk +65720 mike king +65719 bob brown +65719 holly ichabod +65719 ulysses ellison +65719 ulysses young +65718 jessica ichabod +65718 oscar johnson +65718 victor hernandez +65718 zach ovid +65717 holly hernandez +65717 mike nixon +65717 quinn garcia +65717 ulysses davidson +65717 ulysses polk +65716 ethan zipper +65716 holly xylophone +65716 jessica garcia +65716 nick falkner +65716 sarah king +65716 victor xylophone +65715 alice hernandez +65715 david young +65715 ethan polk +65715 oscar falkner +65715 priscilla brown +65714 NULL +65714 ethan laertes +65714 luke allen +65713 bob ellison +65713 nick nixon +65713 ulysses carson +65713 ulysses ichabod +65713 zach garcia +65712 NULL +65712 katie king +65712 luke davidson +65712 quinn garcia +65712 zach carson +65711 bob zipper +65711 fred miller +65711 holly nixon +65711 katie ellison +65711 wendy ovid +65711 zach zipper +65710 luke polk +65710 mike young +65709 bob laertes +65709 calvin laertes +65709 oscar laertes +65708 gabriella laertes +65708 priscilla van buren +65707 bob garcia +65707 mike steinbeck +65706 bob ellison +65706 bob xylophone +65706 luke allen +65706 ulysses xylophone +65706 wendy ichabod +65705 david ellison +65705 mike white +65705 priscilla johnson +65705 tom ellison +65704 ethan brown +65704 holly king +65704 jessica young +65704 tom steinbeck +65704 victor steinbeck +65704 zach falkner +65703 holly underhill +65703 mike ellison +65703 victor davidson +65703 xavier underhill +65702 NULL +65702 bob brown +65702 bob zipper +65702 ulysses quirinius +65701 alice allen +65701 mike steinbeck +65701 oscar thompson +65700 luke allen +65700 quinn laertes +65700 tom quirinius +65699 ethan brown +65699 ethan van buren +65699 irene laertes +65699 mike young +65699 nick garcia +65699 sarah white +65698 gabriella thompson +65698 nick polk +65697 NULL +65697 NULL +65697 holly miller +65697 oscar quirinius +65697 oscar thompson +65697 xavier davidson +65696 alice xylophone +65696 jessica davidson +65696 luke allen +65696 oscar xylophone +65695 oscar king +65695 rachel young +65695 wendy ellison +65695 yuri quirinius +65694 david brown +65694 holly underhill +65694 victor quirinius +65694 zach brown +65693 bob hernandez +65693 bob young +65693 david brown +65693 holly hernandez +65693 tom polk +65693 ulysses polk +65693 victor brown +65692 holly johnson +65692 tom robinson +65691 calvin ovid +65691 ethan nixon +65691 ethan robinson +65691 fred underhill +65691 holly white +65691 irene polk +65691 oscar white +65691 rachel brown +65690 fred van buren +65690 jessica quirinius +65689 oscar ovid +65689 wendy thompson +65688 bob steinbeck +65688 victor steinbeck +65687 gabriella ichabod +65687 jessica underhill +65687 mike zipper +65687 quinn thompson +65686 bob king +65686 bob zipper +65686 david quirinius +65686 luke ichabod +65685 ethan robinson +65685 gabriella hernandez +65685 katie garcia +65685 sarah ellison +65685 victor hernandez +65685 victor nixon +65684 priscilla brown +65684 victor laertes +65684 wendy van buren +65683 NULL +65683 mike king +65683 tom laertes +65682 calvin quirinius +65682 ethan brown +65682 katie ellison +65681 gabriella allen +65681 luke laertes +65681 oscar quirinius +65681 ulysses laertes +65681 wendy ellison +65681 xavier polk +65680 NULL +65680 alice nixon +65680 gabriella ovid +65680 jessica carson +65680 ulysses nixon +65680 zach zipper +65679 bob garcia +65679 wendy underhill +65678 bob falkner +65678 victor xylophone +65678 wendy king +65677 alice allen +65677 fred van buren +65677 mike brown +65677 nick xylophone +65677 ulysses underhill +65677 zach robinson +65676 bob davidson +65676 bob laertes +65676 tom ovid +65676 xavier johnson +65675 david hernandez +65675 david nixon +65675 holly falkner +65675 quinn steinbeck +65675 rachel robinson +65675 sarah zipper +65675 tom polk +65675 victor allen +65674 gabriella falkner +65673 nick johnson +65673 quinn brown +65673 quinn underhill +65673 rachel ovid +65673 wendy brown +65672 nick laertes +65672 nick underhill +65672 rachel zipper +65672 tom white +65672 victor king +65671 fred ellison +65671 fred falkner +65671 zach white +65670 david robinson +65670 jessica zipper +65670 luke van buren +65670 oscar ovid +65670 quinn steinbeck +65669 NULL +65669 alice king +65669 calvin hernandez +65669 katie polk +65669 nick miller +65669 oscar van buren +65668 luke ellison +65667 bob brown +65667 irene nixon +65667 oscar brown +65667 tom falkner +65666 +65666 david underhill +65666 fred van buren +65665 rachel brown +65664 NULL +65664 bob davidson +65664 david ichabod +65664 ethan laertes +65664 irene robinson +65664 mike carson +65664 priscilla young +65664 victor king +65663 calvin underhill +65663 jessica johnson +65663 priscilla carson +65663 zach ichabod +65662 ethan allen +65662 katie ovid +65662 oscar johnson +65662 ulysses carson +65662 ulysses polk +65662 victor ovid +65661 david van buren +65661 luke xylophone +65661 mike falkner +65661 priscilla van buren +65661 victor johnson +65660 holly ichabod +65660 priscilla johnson +65660 victor thompson +65659 david robinson +65659 gabriella king +65659 luke davidson +65659 mike king +65659 mike zipper +65659 nick brown +65659 nick zipper +65659 yuri underhill +65658 NULL +65658 alice zipper +65658 calvin allen +65658 calvin johnson +65658 jessica garcia +65658 quinn davidson +65658 sarah ovid +65658 ulysses brown +65658 ulysses miller +65658 yuri king +65657 ethan falkner +65657 holly zipper +65657 irene ovid +65657 luke ovid +65657 priscilla white +65656 david davidson +65656 irene ovid +65656 jessica xylophone +65656 luke laertes +65656 oscar ichabod +65656 xavier ellison +65655 calvin falkner +65655 yuri laertes +65654 alice carson +65654 alice quirinius +65654 gabriella falkner +65654 nick young +65654 oscar robinson +65654 quinn robinson +65654 rachel falkner +65654 tom laertes +65654 yuri johnson +65653 calvin ellison +65653 holly underhill +65653 ulysses polk +65653 wendy xylophone +65652 NULL +65652 tom ellison +65652 victor johnson +65651 NULL +65651 ethan laertes +65651 ethan laertes +65651 irene garcia +65651 mike young +65650 irene ellison +65650 oscar white +65650 sarah steinbeck +65650 ulysses underhill +65650 ulysses xylophone +65650 victor xylophone +65649 irene underhill +65649 priscilla quirinius +65649 quinn ellison +65649 tom quirinius +65648 alice nixon +65648 calvin brown +65648 sarah carson +65648 xavier ellison +65647 irene allen +65647 mike ellison +65646 bob ovid +65646 xavier brown +65646 xavier ovid +65645 jessica quirinius +65645 katie miller +65645 ulysses hernandez +65644 alice king +65644 calvin ovid +65644 jessica white +65644 katie van buren +65644 sarah young +65644 ulysses hernandez +65644 yuri carson +65643 david davidson +65643 ethan polk +65643 ethan zipper +65643 gabriella ichabod +65643 mike davidson +65643 mike hernandez +65643 oscar robinson +65643 priscilla underhill +65643 zach king +65642 gabriella thompson +65641 fred laertes +65641 sarah garcia +65641 tom miller +65641 xavier hernandez +65640 david underhill +65639 wendy garcia +65638 fred nixon +65638 luke polk +65638 rachel carson +65637 alice underhill +65637 david davidson +65637 fred davidson +65637 gabriella davidson +65637 oscar carson +65637 rachel laertes +65637 sarah garcia +65637 wendy garcia +65636 +65636 irene polk +65636 wendy allen +65635 alice steinbeck +65635 alice zipper +65635 ulysses white +65634 NULL +65634 calvin white +65634 holly underhill +65634 sarah falkner +65633 NULL +65633 holly polk +65633 jessica nixon +65633 oscar quirinius +65632 alice falkner +65632 zach quirinius +65631 fred ellison +65630 rachel brown +65630 xavier hernandez +65629 jessica quirinius +65629 priscilla carson +65629 victor young +65629 xavier carson +65628 bob ovid +65628 ethan ovid +65628 irene ichabod +65628 oscar hernandez +65628 oscar robinson +65628 xavier quirinius +65627 alice miller +65627 holly johnson +65627 luke falkner +65627 yuri polk +65626 ethan polk +65626 holly robinson +65626 tom young +65626 yuri johnson +65625 david xylophone +65625 fred ichabod +65625 katie white +65625 ulysses garcia +65624 calvin steinbeck +65624 calvin xylophone +65624 rachel carson +65624 tom van buren +65624 yuri brown +65623 alice quirinius +65623 jessica miller +65623 oscar ichabod +65623 quinn zipper +65623 tom van buren +65623 victor brown +65623 wendy young +65622 nick davidson +65622 rachel king +65622 wendy robinson +65622 xavier ichabod +65622 zach xylophone +65622 zach young +65621 quinn underhill +65621 ulysses young +65620 nick garcia +65620 oscar thompson +65620 quinn quirinius +65620 victor white +65620 victor xylophone +65620 wendy quirinius +65619 calvin brown +65619 gabriella polk +65619 oscar king +65619 ulysses miller +65619 ulysses robinson +65619 ulysses steinbeck +65618 gabriella ovid +65618 irene laertes +65618 katie king +65618 oscar ovid +65618 quinn thompson +65617 fred van buren +65617 gabriella carson +65617 sarah johnson +65617 ulysses underhill +65616 calvin steinbeck +65616 xavier ichabod +65615 alice ovid +65615 david quirinius +65615 irene quirinius +65615 katie nixon +65614 wendy king +65614 xavier quirinius +65614 xavier white +65613 xavier zipper +65612 irene miller +65612 victor hernandez +65612 wendy white +65612 yuri polk +65611 ethan johnson +65611 fred zipper +65611 irene carson +65611 nick quirinius +65610 tom king +65610 victor steinbeck +65610 wendy garcia +65610 yuri carson +65610 zach ovid +65609 sarah robinson +65608 katie van buren +65608 mike van buren +65608 quinn ichabod +65608 zach underhill +65607 katie miller +65607 luke falkner +65607 mike polk +65607 priscilla xylophone +65607 yuri allen +65607 yuri allen +65606 bob white +65606 gabriella white +65606 oscar carson +65606 victor white +65606 xavier allen +65606 zach allen +65605 holly king +65604 katie zipper +65604 oscar davidson +65604 wendy laertes +65604 zach brown +65603 alice davidson +65603 ethan miller +65603 katie davidson +65603 katie young +65603 mike garcia +65602 NULL +65602 calvin laertes +65602 ethan laertes +65602 fred steinbeck +65602 jessica young +65602 xavier brown +65601 priscilla ovid +65601 sarah xylophone +65601 tom robinson +65600 gabriella thompson +65600 jessica polk +65600 nick robinson +65600 rachel allen +65599 fred quirinius +65599 luke johnson +65599 nick garcia +65599 oscar xylophone +65599 ulysses underhill +65598 ulysses van buren +65598 victor zipper +65597 ethan ellison +65597 nick ellison +65597 quinn davidson +65596 NULL +65596 calvin zipper +65596 david ellison +65596 irene ichabod +65596 wendy laertes +65595 bob white +65595 holly hernandez +65595 luke brown +65595 oscar ellison +65595 oscar ichabod +65595 quinn ellison +65594 gabriella ellison +65594 oscar robinson +65594 ulysses underhill +65594 victor robinson +65593 oscar white +65593 zach xylophone +65592 calvin xylophone +65591 alice zipper +65591 nick ichabod +65591 priscilla ichabod +65591 rachel underhill +65590 NULL +65590 katie falkner +65590 oscar van buren +65590 xavier garcia +65590 yuri underhill +65589 ethan white +65589 gabriella zipper +65589 irene ovid +65589 oscar king +65589 wendy xylophone +65588 bob van buren +65588 david ichabod +65588 mike miller +65588 tom hernandez +65588 victor van buren +65587 bob garcia +65587 luke johnson +65587 mike king +65587 victor allen +65587 xavier white +65586 david young +65586 irene brown +65586 priscilla brown +65586 wendy allen +65586 xavier laertes +65585 alice garcia +65585 bob garcia +65585 ethan ellison +65585 nick ellison +65585 priscilla thompson +65584 jessica carson +65584 jessica van buren +65584 jessica white +65583 bob xylophone +65583 nick ichabod +65583 yuri brown +65583 yuri steinbeck +65582 holly johnson +65582 mike carson +65582 victor van buren +65582 zach miller +65581 gabriella steinbeck +65581 irene quirinius +65581 luke allen +65581 nick robinson +65581 nick young +65581 wendy robinson +65580 alice steinbeck +65580 alice xylophone +65580 irene xylophone +65579 irene polk +65579 luke ovid +65579 quinn nixon +65579 sarah garcia +65579 wendy ovid +65578 calvin robinson +65578 fred king +65578 holly thompson +65578 katie ichabod +65578 quinn king +65578 rachel davidson +65578 victor hernandez +65577 holly white +65576 calvin falkner +65576 calvin ovid +65576 fred polk +65576 luke robinson +65575 calvin falkner +65575 irene steinbeck +65575 luke zipper +65575 zach king +65574 gabriella steinbeck +65574 priscilla nixon +65574 rachel thompson +65573 victor ellison +65573 victor hernandez +65573 yuri nixon +65572 calvin davidson +65572 calvin young +65572 katie young +65572 oscar ellison +65572 quinn garcia +65571 bob king +65571 irene polk +65571 katie ichabod +65571 mike steinbeck +65570 NULL +65570 bob ovid +65570 fred polk +65570 luke ellison +65570 mike hernandez +65570 yuri quirinius +65569 nick falkner +65568 bob ichabod +65568 holly thompson +65568 jessica thompson +65567 katie xylophone +65566 gabriella garcia +65566 rachel white +65565 katie young +65565 quinn young +65564 alice polk +65564 calvin carson +65564 calvin white +65564 ethan hernandez +65564 ethan quirinius +65564 jessica thompson +65564 katie hernandez +65563 calvin zipper +65563 priscilla young +65563 xavier davidson +65563 yuri steinbeck +65562 calvin falkner +65562 ethan xylophone +65562 luke white +65562 quinn allen +65562 rachel polk +65562 wendy polk +65561 bob davidson +65561 ethan polk +65561 jessica robinson +65560 fred white +65560 jessica johnson +65560 oscar thompson +65560 ulysses steinbeck +65560 zach brown +65559 NULL +65559 ethan laertes +65559 gabriella ichabod +65559 gabriella zipper +65559 irene garcia +65558 fred robinson +65557 fred hernandez +65557 nick johnson +65556 oscar underhill +65556 xavier hernandez +65556 yuri falkner +65556 zach garcia +65556 zach steinbeck +65555 fred nixon +65554 gabriella miller +65554 rachel falkner +65553 calvin van buren +65553 david van buren +65553 irene nixon +65553 luke laertes +65553 oscar carson +65552 NULL +65552 irene ellison +65552 oscar polk +65552 wendy falkner +65552 zach miller +65551 fred young +65551 ulysses underhill +65551 wendy underhill +65550 ethan quirinius +65550 fred davidson +65550 holly young +65550 jessica ovid +65550 quinn brown +65550 quinn laertes +65550 tom johnson +65549 bob garcia +65549 bob ovid +65549 fred ichabod +65549 fred king +65549 jessica white +65549 ulysses laertes +65549 victor davidson +65549 victor miller +65548 calvin ovid +65548 gabriella allen +65548 holly ichabod +65548 priscilla johnson +65548 quinn zipper +65548 tom hernandez +65548 wendy king +65547 bob ellison +65547 jessica quirinius +65547 mike davidson +65547 xavier allen +65546 katie white +65545 mike king +65545 tom carson +65545 victor brown +65544 calvin davidson +65544 calvin nixon +65544 david ovid +65544 irene thompson +65544 ulysses young +65544 xavier polk +65544 xavier xylophone +65544 zach ovid +65543 fred johnson +65543 sarah johnson +65542 fred falkner +65542 holly thompson +65542 luke miller +65542 mike white +65542 tom davidson +65541 calvin brown +65541 ethan brown +65541 holly brown +65541 jessica falkner +65541 rachel thompson +65541 tom zipper +65541 wendy underhill +65541 xavier polk +65541 yuri johnson +65540 rachel falkner +65539 gabriella young +65539 holly laertes +65539 oscar carson +65538 irene laertes +65538 mike polk +65538 tom robinson +65537 NULL +65537 david quirinius +65537 rachel ovid +65537 ulysses laertes +65537 zach garcia +65536 calvin xylophone +65536 david thompson +65536 irene falkner +65536 ulysses johnson +65536 victor johnson +65536 wendy miller +65536 yuri thompson diff --git a/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-2-1e88e0ba414a00195f7ebf6b8600ac04 b/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-2-1e88e0ba414a00195f7ebf6b8600ac04 new file mode 100644 index 0000000000000..62d71abc6fc7d --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-2-1e88e0ba414a00195f7ebf6b8600ac04 @@ -0,0 +1,1049 @@ +65536 NULL +65536 16.85 +65536 32.68 +65536 33.45 +65536 58.86 +65536 75.7 +65536 83.48 +65537 NULL +65537 4.49 +65537 11.87 +65537 51.91 +65537 99.34 +65538 NULL +65538 9.53 +65538 30.27 +65539 NULL +65539 58.85 +65539 96.64 +65540 NULL +65541 NULL +65541 9.04 +65541 14.94 +65541 15.85 +65541 27.89 +65541 35.38 +65541 72.33 +65541 89.14 +65541 98.87 +65542 NULL +65542 42.85 +65542 73.72 +65542 73.93 +65542 84.63 +65543 NULL +65543 21.59 +65544 NULL +65544 26.94 +65544 32.03 +65544 48.84 +65544 65.77 +65544 65.85 +65544 73.08 +65544 79.21 +65545 NULL +65545 34.65 +65545 65.81 +65546 NULL +65547 NULL +65547 17.71 +65547 62.31 +65547 83.21 +65548 NULL +65548 52.94 +65548 53.52 +65548 57.61 +65548 58.51 +65548 75.39 +65548 77.24 +65549 NULL +65549 13.3 +65549 28.93 +65549 50.6 +65549 55.04 +65549 64.91 +65549 76.06 +65549 80.09 +65550 NULL +65550 1.75 +65550 8.46 +65550 33.01 +65550 57.63 +65550 91.38 +65550 96.9 +65551 NULL +65551 39.43 +65551 73.93 +65552 NULL +65552 3.43 +65552 18.11 +65552 48.96 +65552 82.21 +65553 NULL +65553 25.31 +65553 29.62 +65553 71.07 +65553 72.16 +65554 NULL +65554 0.24 +65555 NULL +65556 NULL +65556 9.96 +65556 45.81 +65556 50.42 +65556 95.12 +65557 NULL +65557 21.14 +65558 NULL +65559 NULL +65559 29.55 +65559 56.06 +65559 73.94 +65559 83.5 +65560 NULL +65560 16.86 +65560 21.81 +65560 64.38 +65560 71.59 +65561 NULL +65561 32.86 +65561 47.71 +65562 NULL +65562 26.86 +65562 45.94 +65562 54.94 +65562 69.41 +65562 72.04 +65563 NULL +65563 14.36 +65563 33.29 +65563 39.96 +65564 NULL +65564 9.47 +65564 15.82 +65564 26.97 +65564 30.78 +65564 90.34 +65564 98.36 +65565 NULL +65565 81.72 +65566 NULL +65566 7.8 +65567 NULL +65568 NULL +65568 21.79 +65568 58.66 +65569 NULL +65570 NULL +65570 17.09 +65570 18.2 +65570 25.57 +65570 45.23 +65570 76.8 +65571 NULL +65571 26.64 +65571 40.68 +65571 82.5 +65572 NULL +65572 22.64 +65572 43.49 +65572 70.91 +65572 93.48 +65573 NULL +65573 53.56 +65573 96.32 +65574 NULL +65574 31.28 +65574 38.54 +65575 NULL +65575 17 +65575 32.85 +65575 83.4 +65576 NULL +65576 2.04 +65576 4.88 +65576 66.55 +65577 NULL +65578 NULL +65578 16.01 +65578 41.1 +65578 51.36 +65578 54.35 +65578 58.78 +65578 75.62 +65579 NULL +65579 21.36 +65579 33.37 +65579 73.48 +65579 91.42 +65580 NULL +65580 56.33 +65580 81.42 +65581 NULL +65581 29.74 +65581 45.48 +65581 56.59 +65581 60.88 +65581 88.09 +65582 NULL +65582 1.23 +65582 9.35 +65582 96.6 +65583 NULL +65583 28.07 +65583 50.57 +65583 57.67 +65584 NULL +65584 6.88 +65584 65.78 +65585 NULL +65585 31.23 +65585 37.34 +65585 39.32 +65585 50.38 +65586 NULL +65586 1.15 +65586 18.65 +65586 87.57 +65586 94.25 +65587 NULL +65587 5.83 +65587 11.86 +65587 53.84 +65587 94.47 +65588 NULL +65588 64.63 +65588 69.48 +65588 76.44 +65588 98.33 +65589 NULL +65589 49.49 +65589 72.3 +65589 74.83 +65589 94.73 +65590 NULL +65590 13.51 +65590 22.13 +65590 42.53 +65590 92.71 +65591 NULL +65591 9.85 +65591 11.43 +65591 60.78 +65592 NULL +65593 NULL +65593 35.15 +65594 NULL +65594 35.28 +65594 60.57 +65594 65.61 +65595 NULL +65595 8.76 +65595 67.56 +65595 72.7 +65595 89.6 +65595 90.24 +65596 NULL +65596 12.72 +65596 54.75 +65596 85.74 +65596 94.35 +65597 NULL +65597 37.41 +65597 69.05 +65598 NULL +65598 63.3 +65599 NULL +65599 0.56 +65599 4.93 +65599 41.61 +65599 76.29 +65600 NULL +65600 36.57 +65600 43.03 +65600 92.92 +65601 NULL +65601 26.54 +65601 37.93 +65602 NULL +65602 8.13 +65602 47.16 +65602 83.67 +65602 86.23 +65602 95.58 +65603 NULL +65603 41.44 +65603 45.63 +65603 69.26 +65603 80.24 +65604 NULL +65604 63.36 +65604 63.65 +65604 85.91 +65605 NULL +65606 NULL +65606 7.51 +65606 24.8 +65606 57.69 +65606 67.94 +65606 87.16 +65607 NULL +65607 9.67 +65607 36.58 +65607 71.75 +65607 75.86 +65607 91.52 +65608 NULL +65608 48.9 +65608 69.42 +65608 87.9 +65609 NULL +65610 NULL +65610 7.59 +65610 11.99 +65610 36.77 +65610 39.74 +65611 NULL +65611 21.21 +65611 25.92 +65611 64.89 +65612 NULL +65612 16.05 +65612 25.1 +65612 52.64 +65613 NULL +65614 NULL +65614 1.42 +65614 94.47 +65615 NULL +65615 10.79 +65615 39.4 +65615 99.88 +65616 NULL +65616 75.2 +65617 NULL +65617 18.51 +65617 47.45 +65617 64.9 +65618 NULL +65618 10.06 +65618 16.6 +65618 81.99 +65618 88.38 +65619 NULL +65619 27.32 +65619 32.64 +65619 34.72 +65619 36.48 +65619 36.59 +65620 NULL +65620 6.85 +65620 8.16 +65620 29.14 +65620 64.65 +65620 81.28 +65621 NULL +65621 95.14 +65622 NULL +65622 28.37 +65622 50.08 +65622 74.31 +65622 88.6 +65622 93.7 +65623 NULL +65623 30.83 +65623 31.22 +65623 39.74 +65623 48.51 +65623 95.58 +65623 97.2 +65624 NULL +65624 58.02 +65624 65.31 +65624 70.08 +65624 93.3 +65625 NULL +65625 20.61 +65625 42.86 +65625 55.06 +65626 NULL +65626 63.54 +65626 64.61 +65626 75.15 +65627 NULL +65627 19.65 +65627 61.89 +65627 93.29 +65628 NULL +65628 14.83 +65628 30.43 +65628 37.8 +65628 74.31 +65628 83.26 +65629 NULL +65629 19.33 +65629 58.81 +65629 72.9 +65630 NULL +65630 72.13 +65631 NULL +65632 NULL +65632 88.51 +65633 NULL +65633 59.56 +65633 72.54 +65633 81.02 +65634 NULL +65634 57.09 +65634 64.36 +65634 99.34 +65635 NULL +65635 64.99 +65635 82.29 +65636 NULL +65636 21.15 +65636 86.29 +65637 NULL +65637 16.89 +65637 26.78 +65637 29.34 +65637 35.51 +65637 44.32 +65637 48.88 +65637 93.41 +65638 NULL +65638 11.2 +65638 19.13 +65639 NULL +65640 NULL +65641 NULL +65641 26.02 +65641 84.27 +65641 91.46 +65642 NULL +65643 NULL +65643 22.05 +65643 50.79 +65643 52.56 +65643 61.29 +65643 71.29 +65643 80.96 +65643 92.24 +65643 93.11 +65644 NULL +65644 1.97 +65644 30.25 +65644 58.05 +65644 87.31 +65644 89.95 +65644 96.45 +65645 NULL +65645 3.95 +65645 63.22 +65646 NULL +65646 17.92 +65646 27.34 +65647 NULL +65647 58.03 +65648 NULL +65648 0.08 +65648 17.66 +65648 64.06 +65649 NULL +65649 8.69 +65649 43.92 +65649 91.03 +65650 NULL +65650 23.55 +65650 59.55 +65650 85.89 +65650 89.12 +65650 90.77 +65651 NULL +65651 24.25 +65651 58.25 +65651 74.13 +65651 84.42 +65652 NULL +65652 55.04 +65652 73.61 +65653 NULL +65653 3.81 +65653 52.23 +65653 85.09 +65654 NULL +65654 8.91 +65654 11.64 +65654 26.73 +65654 29.85 +65654 37.74 +65654 37.8 +65654 53.55 +65654 88.23 +65655 NULL +65655 77.41 +65656 NULL +65656 14 +65656 14.96 +65656 53.27 +65656 64.44 +65656 82.67 +65657 NULL +65657 11.93 +65657 26.4 +65657 64.39 +65657 65.01 +65658 NULL +65658 2.63 +65658 20.69 +65658 42.93 +65658 46.61 +65658 60.94 +65658 66.53 +65658 68.85 +65658 77.66 +65658 92.67 +65659 NULL +65659 8.95 +65659 46.57 +65659 53.8 +65659 94.3 +65659 94.69 +65659 95.71 +65659 99.87 +65660 NULL +65660 28.05 +65660 62.82 +65661 NULL +65661 5.24 +65661 8.06 +65661 26.8 +65661 68.98 +65662 NULL +65662 59.92 +65662 76.11 +65662 76.51 +65662 88.64 +65662 99.18 +65663 NULL +65663 5.42 +65663 78.56 +65663 94.16 +65664 NULL +65664 11.46 +65664 27.6 +65664 34.71 +65664 38.42 +65664 45.4 +65664 55.82 +65664 97.64 +65665 NULL +65666 NULL +65666 32.73 +65666 83.95 +65667 NULL +65667 13.96 +65667 63.9 +65667 97.87 +65668 NULL +65669 NULL +65669 1.76 +65669 16.95 +65669 38.6 +65669 54.25 +65669 93.79 +65670 NULL +65670 5.37 +65670 61.06 +65670 61.54 +65670 92.97 +65671 NULL +65671 8.65 +65671 52.05 +65672 NULL +65672 52.6 +65672 58.1 +65672 64.09 +65672 75.27 +65673 NULL +65673 0.9 +65673 33.27 +65673 43.81 +65673 87.78 +65674 NULL +65675 NULL +65675 4.19 +65675 24.19 +65675 35.33 +65675 35.78 +65675 79.9 +65675 83.09 +65675 87.36 +65676 NULL +65676 8.77 +65676 58.12 +65676 80.13 +65677 NULL +65677 5.06 +65677 25.37 +65677 44.47 +65677 48.79 +65677 87.67 +65678 NULL +65678 8.72 +65678 33.9 +65679 NULL +65679 64.15 +65680 NULL +65680 1.01 +65680 34.08 +65680 54.11 +65680 55.3 +65680 65.88 +65681 NULL +65681 35.45 +65681 41.57 +65681 61.3 +65681 71.17 +65681 75.85 +65682 NULL +65682 67.17 +65682 92.95 +65683 NULL +65683 17.62 +65683 99.56 +65684 NULL +65684 3.51 +65684 67.34 +65685 NULL +65685 38.71 +65685 43.48 +65685 63.27 +65685 87.84 +65685 90.69 +65686 NULL +65686 31.75 +65686 58.87 +65686 98.68 +65687 NULL +65687 3.37 +65687 21.79 +65687 48.73 +65688 NULL +65688 76.21 +65689 NULL +65689 9.12 +65690 NULL +65690 3.43 +65691 NULL +65691 5.01 +65691 6.93 +65691 28.47 +65691 56.02 +65691 58.01 +65691 69.8 +65691 76.98 +65692 NULL +65692 54.76 +65693 NULL +65693 8.38 +65693 32.33 +65693 45.69 +65693 69.32 +65693 71.72 +65693 84.88 +65694 NULL +65694 58.23 +65694 82.24 +65694 88.5 +65695 NULL +65695 57.33 +65695 59.96 +65695 77.09 +65696 NULL +65696 17.35 +65696 40.3 +65696 54.02 +65697 NULL +65697 3.18 +65697 50.01 +65697 67.9 +65697 86.79 +65697 90.16 +65698 NULL +65698 42.98 +65699 NULL +65699 13.29 +65699 38.71 +65699 68.94 +65699 84.79 +65699 88.09 +65700 NULL +65700 2.83 +65700 37.61 +65701 NULL +65701 1.81 +65701 6.35 +65702 NULL +65702 37.6 +65702 55.68 +65702 79.5 +65703 NULL +65703 37.18 +65703 40.81 +65703 90.89 +65704 NULL +65704 16.22 +65704 37.12 +65704 48.48 +65704 54.76 +65704 93.21 +65705 NULL +65705 20.57 +65705 25.89 +65705 65.13 +65706 NULL +65706 3.91 +65706 9.74 +65706 55.94 +65706 72.87 +65707 NULL +65707 76.2 +65708 NULL +65708 1.29 +65709 NULL +65709 5.64 +65709 49.79 +65710 NULL +65710 86.7 +65711 NULL +65711 8.66 +65711 50.26 +65711 71.89 +65711 78.69 +65711 96.1 +65712 NULL +65712 30.27 +65712 34.7 +65712 49.69 +65712 53.65 +65713 NULL +65713 10.94 +65713 39.47 +65713 72.37 +65713 90.91 +65714 NULL +65714 14.85 +65714 47.42 +65715 NULL +65715 39.62 +65715 54.79 +65715 81.28 +65715 89.4 +65716 NULL +65716 9 +65716 10.07 +65716 33.4 +65716 71.53 +65716 85.93 +65717 NULL +65717 1.23 +65717 5.81 +65717 57.61 +65717 80.05 +65718 NULL +65718 63.06 +65718 84.35 +65718 89.67 +65719 NULL +65719 51.13 +65719 66.85 +65719 82.1 +65720 NULL +65720 2.72 +65720 18.8 +65720 22.34 +65720 62.04 +65721 NULL +65721 23.78 +65721 39.19 +65721 55.75 +65721 72.82 +65721 95.12 +65721 95.38 +65722 NULL +65722 1.76 +65722 38.82 +65723 NULL +65723 39.9 +65724 NULL +65724 10.52 +65724 36.05 +65724 50.96 +65724 71.66 +65724 85.52 +65725 NULL +65726 NULL +65726 6 +65726 60.46 +65727 NULL +65727 19.81 +65727 49.19 +65727 87.37 +65727 88.11 +65728 NULL +65728 55.37 +65729 NULL +65730 NULL +65730 1.35 +65730 30.6 +65730 81.44 +65731 NULL +65731 24.48 +65731 61.52 +65731 97.18 +65732 NULL +65732 30.06 +65732 91.15 +65733 NULL +65733 11.44 +65733 20.72 +65733 88.46 +65733 93.45 +65733 99.8 +65734 NULL +65734 31.71 +65735 NULL +65735 12.67 +65735 61.16 +65736 NULL +65736 28.9 +65736 48.54 +65736 86.51 +65737 NULL +65737 3.98 +65737 20.85 +65737 29.92 +65737 80.97 +65738 NULL +65738 30.94 +65738 82.32 +65738 95.1 +65739 NULL +65739 74.77 +65739 92.4 +65740 NULL +65740 7.49 +65740 58.65 +65741 NULL +65742 NULL +65742 6.61 +65742 43.84 +65743 NULL +65743 26.6 +65743 52.65 +65743 62 +65744 NULL +65744 46.98 +65745 NULL +65745 25.19 +65745 66.36 +65745 80.12 +65746 NULL +65746 36.74 +65746 93.21 +65746 97.52 +65746 98.1 +65747 NULL +65747 11.16 +65747 15.07 +65747 21.8 +65747 39.77 +65747 52.77 +65747 71.87 +65748 NULL +65748 29.49 +65749 NULL +65749 15.14 +65749 45 +65749 65.49 +65749 73.24 +65750 NULL +65750 20.91 +65750 83.44 +65750 85.44 +65750 96.85 +65751 NULL +65751 2.96 +65751 9.02 +65751 30.68 +65751 47.81 +65751 78.75 +65752 NULL +65752 47.82 +65753 NULL +65753 86.97 +65754 NULL +65754 54.35 +65755 NULL +65755 11.23 +65755 22.44 +65755 64 +65755 67.54 +65755 76.75 +65755 81.44 +65755 90.08 +65755 96.8 +65756 NULL +65756 1.45 +65756 11.81 +65756 63.51 +65757 NULL +65757 1.86 +65757 9.24 +65757 34.84 +65757 90.09 +65758 NULL +65758 25.62 +65758 56.56 +65758 60.88 +65758 94.9 +65759 NULL +65759 10.63 +65759 14.1 +65759 47.54 +65759 92.81 +65760 NULL +65760 21.14 +65760 27.52 +65760 95.45 +65761 NULL +65762 NULL +65762 5.49 +65762 45.7 +65762 77.96 +65762 87.5 +65763 NULL +65763 0.72 +65763 43.8 +65763 86.43 +65763 87.99 +65764 NULL +65764 31.41 +65764 57.1 +65765 NULL +65765 88.52 +65765 88.56 +65766 NULL +65766 37.06 +65766 66.34 +65766 86.53 +65766 98.9 +65767 NULL +65767 90.88 +65767 95.57 +65768 NULL +65769 NULL +65769 11.45 +65769 38.98 +65769 58.05 +65769 70.52 +65769 91.49 +65770 NULL +65770 51.9 +65771 NULL +65771 6.15 +65771 7.5 +65772 NULL +65773 NULL +65773 3.81 +65773 18.2 +65773 30.49 +65773 47.09 +65773 53.09 +65773 63.26 +65773 76.46 +65774 NULL +65774 45.74 +65774 45.97 +65774 48.8 +65774 56.84 +65774 94.77 +65775 NULL +65775 7.88 +65775 66.56 +65775 66.68 +65775 98.43 +65776 NULL +65776 18.7 +65776 28.47 +65776 49.73 +65776 98.87 +65777 NULL +65777 54.39 +65777 73.79 +65777 82.62 +65778 NULL +65778 7.37 +65778 51.64 +65778 59.03 +65778 62.17 +65778 64.69 +65778 89.51 +65778 95.69 +65779 NULL +65779 11.87 +65779 28.2 +65779 39.48 +65779 45.61 +65779 64.41 +65779 65.24 +65780 NULL +65780 10.95 +65780 38.58 +65781 NULL +65781 70.59 +65781 95.52 +65782 NULL +65782 30.24 +65782 34.31 +65782 76.14 +65782 81.9 +65783 NULL +65783 46.34 +65783 51.08 +65783 52.43 +65783 62.58 +65783 77.4 +65784 NULL +65784 15.7 +65784 31.35 +65784 68.18 +65784 93.95 +65785 NULL +65785 29.61 +65786 NULL +65786 8.99 +65786 29.32 +65786 66.89 +65786 80.94 +65787 NULL +65787 18.78 +65787 31.19 +65787 64.88 +65788 NULL +65788 16.1 +65788 21.81 +65788 25.77 +65789 NULL +65789 20.44 +65789 43.53 +65789 52.49 +65789 83.18 +65789 92.74 +65789 96.9 +65790 NULL +65790 46.91 +65790 84.87 +65791 NULL +65791 4.24 diff --git a/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-3-34d9ee4120f21d0d0ae914fba0acc60c b/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-3-34d9ee4120f21d0d0ae914fba0acc60c new file mode 100644 index 0000000000000..569c1d4e5f7b6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-3-34d9ee4120f21d0d0ae914fba0acc60c @@ -0,0 +1,1049 @@ + 7 + 11 + 38 +alice allen 48 +alice allen 78 +alice allen 79 +alice brown 91 +alice carson 28 +alice davidson 88 +alice falkner 117 +alice garcia 106 +alice hernandez 37 +alice hernandez 85 +alice johnson 65 +alice king 109 +alice king 113 +alice king 118 +alice laertes 6 +alice laertes 65 +alice miller 97 +alice nixon 15 +alice nixon 31 +alice nixon 56 +alice ovid 15 +alice polk 90 +alice quirinius 27 +alice quirinius 89 +alice robinson 5 +alice robinson 68 +alice steinbeck 45 +alice steinbeck 50 +alice steinbeck 77 +alice underhill 34 +alice van buren 31 +alice xylophone 0 +alice xylophone 5 +alice xylophone 113 +alice zipper 19 +alice zipper 51 +alice zipper 74 +bob brown 35 +bob brown 61 +bob brown 71 +bob carson 40 +bob davidson 44 +bob davidson 103 +bob davidson 124 +bob ellison 7 +bob ellison 46 +bob ellison 85 +bob ellison 103 +bob falkner 2 +bob garcia 16 +bob garcia 42 +bob garcia 71 +bob garcia 77 +bob garcia 83 +bob hernandez 93 +bob ichabod 72 +bob king 9 +bob king 22 +bob king 81 +bob laertes -1 +bob laertes 105 +bob miller 31 +bob ovid 26 +bob ovid 27 +bob ovid 81 +bob ovid 86 +bob polk 55 +bob quirinius 26 +bob steinbeck 95 +bob van buren 88 +bob white 1 +bob white 16 +bob xylophone -2 +bob xylophone 49 +bob young -1 +bob zipper 36 +bob zipper 78 +bob zipper 92 +calvin allen 98 +calvin brown 81 +calvin brown 87 +calvin brown 121 +calvin carson 105 +calvin davidson 34 +calvin davidson 89 +calvin ellison 34 +calvin falkner -1 +calvin falkner 12 +calvin falkner 46 +calvin falkner 50 +calvin falkner 53 +calvin falkner 101 +calvin garcia 79 +calvin hernandez 22 +calvin johnson 34 +calvin laertes 37 +calvin laertes 100 +calvin nixon 50 +calvin nixon 71 +calvin nixon 72 +calvin ovid -1 +calvin ovid 50 +calvin ovid 65 +calvin ovid 71 +calvin polk 23 +calvin quirinius 5 +calvin quirinius 96 +calvin robinson 0 +calvin steinbeck 13 +calvin steinbeck 14 +calvin steinbeck 33 +calvin thompson 12 +calvin thompson 40 +calvin underhill 19 +calvin van buren 57 +calvin van buren 78 +calvin white 26 +calvin white 80 +calvin xylophone 17 +calvin xylophone 48 +calvin xylophone 78 +calvin young 8 +calvin young 99 +calvin zipper 31 +calvin zipper 46 +david allen 27 +david allen 80 +david brown 75 +david brown 117 +david davidson 11 +david davidson 38 +david davidson 54 +david davidson 74 +david ellison 50 +david ellison 54 +david ellison 120 +david hernandez 72 +david ichabod 6 +david ichabod 55 +david laertes 88 +david nixon 118 +david ovid 61 +david ovid 95 +david quirinius 43 +david quirinius 85 +david quirinius 121 +david robinson 47 +david robinson 59 +david thompson 89 +david underhill 69 +david underhill 87 +david underhill 98 +david van buren 1 +david van buren 38 +david white 93 +david xylophone 0 +david xylophone 22 +david xylophone 82 +david young 2 +david young 79 +ethan allen 24 +ethan brown 3 +ethan brown 29 +ethan brown 55 +ethan brown 64 +ethan brown 84 +ethan brown 108 +ethan carson 83 +ethan ellison 28 +ethan ellison 65 +ethan falkner 45 +ethan falkner 49 +ethan garcia 16 +ethan hernandez 71 +ethan johnson 108 +ethan king 44 +ethan laertes -1 +ethan laertes 27 +ethan laertes 46 +ethan laertes 68 +ethan laertes 81 +ethan laertes 103 +ethan laertes 114 +ethan miller 40 +ethan nixon 18 +ethan ovid 66 +ethan polk 46 +ethan polk 71 +ethan polk 114 +ethan polk 117 +ethan quirinius 16 +ethan quirinius 41 +ethan quirinius 85 +ethan robinson 32 +ethan robinson 34 +ethan underhill 89 +ethan van buren 43 +ethan white 38 +ethan white 51 +ethan xylophone 107 +ethan zipper 37 +ethan zipper 89 +fred davidson 18 +fred davidson 18 +fred davidson 77 +fred ellison -3 +fred ellison 44 +fred ellison 116 +fred falkner 18 +fred falkner 33 +fred falkner 100 +fred hernandez 36 +fred ichabod 1 +fred ichabod 77 +fred johnson 62 +fred king 33 +fred king 92 +fred laertes 17 +fred miller NULL +fred nixon 33 +fred nixon 36 +fred nixon 102 +fred nixon 111 +fred polk -2 +fred polk 39 +fred polk 60 +fred polk 85 +fred quirinius 25 +fred quirinius 124 +fred robinson 89 +fred steinbeck 79 +fred steinbeck 118 +fred steinbeck 119 +fred underhill 122 +fred van buren 4 +fred van buren 24 +fred van buren 63 +fred van buren 106 +fred white 97 +fred young 33 +fred young 103 +fred zipper 66 +gabriella allen 114 +gabriella allen 119 +gabriella brown 25 +gabriella brown 92 +gabriella carson 112 +gabriella davidson 45 +gabriella ellison 21 +gabriella ellison 101 +gabriella falkner 14 +gabriella falkner 66 +gabriella falkner 77 +gabriella garcia 110 +gabriella hernandez 20 +gabriella hernandez 36 +gabriella ichabod 17 +gabriella ichabod 66 +gabriella ichabod 71 +gabriella ichabod 90 +gabriella ichabod 91 +gabriella king 100 +gabriella king 115 +gabriella laertes 50 +gabriella miller 35 +gabriella ovid 38 +gabriella ovid 89 +gabriella polk 42 +gabriella polk 105 +gabriella steinbeck 18 +gabriella steinbeck 115 +gabriella thompson 45 +gabriella thompson 70 +gabriella thompson 88 +gabriella van buren 5 +gabriella van buren 117 +gabriella white 37 +gabriella young 48 +gabriella young 107 +gabriella zipper 57 +gabriella zipper 71 +holly allen 63 +holly brown 50 +holly brown 117 +holly falkner NULL +holly hernandez 31 +holly hernandez 43 +holly hernandez 48 +holly hernandez 100 +holly ichabod 28 +holly ichabod 53 +holly ichabod 83 +holly johnson 60 +holly johnson 112 +holly johnson 121 +holly king 90 +holly king 114 +holly laertes 13 +holly miller 28 +holly nixon -2 +holly nixon 120 +holly polk 54 +holly polk 124 +holly robinson 1 +holly thompson 28 +holly thompson 33 +holly thompson 83 +holly underhill 5 +holly underhill 31 +holly underhill 101 +holly underhill 113 +holly van buren 19 +holly white 18 +holly white 90 +holly xylophone 18 +holly young 32 +holly young 72 +holly zipper 78 +holly zipper 88 +irene allen 56 +irene brown 78 +irene brown 93 +irene brown 108 +irene carson 0 +irene ellison -3 +irene ellison 40 +irene falkner 13 +irene falkner 104 +irene garcia 34 +irene garcia 62 +irene garcia 124 +irene ichabod 83 +irene ichabod 112 +irene johnson 88 +irene laertes 9 +irene laertes 54 +irene laertes 60 +irene miller 108 +irene nixon -1 +irene nixon 12 +irene nixon 101 +irene ovid 26 +irene ovid 32 +irene ovid 53 +irene polk 18 +irene polk 92 +irene polk 99 +irene polk 109 +irene polk 116 +irene quirinius 7 +irene quirinius 76 +irene quirinius 97 +irene robinson 51 +irene steinbeck 46 +irene thompson 10 +irene underhill 27 +irene underhill 63 +irene van buren 17 +irene van buren 104 +irene xylophone 18 +jessica brown 117 +jessica carson 3 +jessica carson 13 +jessica carson 88 +jessica davidson 11 +jessica davidson 28 +jessica davidson 89 +jessica davidson 124 +jessica ellison 38 +jessica ellison 50 +jessica falkner 71 +jessica garcia 25 +jessica garcia 43 +jessica ichabod 104 +jessica johnson 31 +jessica johnson 69 +jessica miller 74 +jessica nixon 22 +jessica nixon 120 +jessica ovid 47 +jessica ovid 73 +jessica polk 118 +jessica quirinius 0 +jessica quirinius 87 +jessica quirinius 105 +jessica quirinius 114 +jessica robinson 15 +jessica thompson 1 +jessica thompson 77 +jessica underhill 32 +jessica underhill 46 +jessica underhill 83 +jessica van buren 54 +jessica white 5 +jessica white 30 +jessica white 45 +jessica white 65 +jessica white 98 +jessica xylophone 67 +jessica young 61 +jessica young 123 +jessica zipper 27 +jessica zipper 33 +jessica zipper 54 +katie allen 114 +katie brown 39 +katie davidson 35 +katie ellison 5 +katie ellison 58 +katie falkner 15 +katie garcia 49 +katie garcia 65 +katie hernandez 83 +katie ichabod 9 +katie ichabod 75 +katie ichabod 104 +katie king 44 +katie king 59 +katie king 93 +katie miller 23 +katie miller 117 +katie nixon 43 +katie ovid 81 +katie polk 17 +katie polk 85 +katie robinson 92 +katie van buren 25 +katie van buren 88 +katie white 34 +katie white 86 +katie xylophone 84 +katie young 2 +katie young 24 +katie young 70 +katie zipper 25 +katie zipper 87 +luke allen 7 +luke allen 44 +luke allen 62 +luke allen 100 +luke allen 114 +luke brown 112 +luke davidson 51 +luke davidson 84 +luke ellison 35 +luke ellison 40 +luke ellison 86 +luke falkner 59 +luke falkner 97 +luke garcia 51 +luke garcia 100 +luke ichabod 42 +luke ichabod 123 +luke johnson 9 +luke johnson 17 +luke johnson 53 +luke laertes 66 +luke laertes 73 +luke laertes 76 +luke laertes 101 +luke laertes 118 +luke miller 93 +luke ovid 43 +luke ovid 70 +luke polk 53 +luke polk 88 +luke quirinius 82 +luke robinson 0 +luke robinson 114 +luke thompson 51 +luke underhill 2 +luke underhill 109 +luke underhill 119 +luke van buren 43 +luke white 110 +luke xylophone 15 +luke zipper 10 +mike allen 0 +mike brown 88 +mike carson 12 +mike carson 17 +mike carson 122 +mike davidson 9 +mike davidson 110 +mike ellison 5 +mike ellison 50 +mike ellison 70 +mike ellison 94 +mike ellison 95 +mike falkner 61 +mike garcia 2 +mike garcia 68 +mike garcia 110 +mike hernandez 91 +mike hernandez 106 +mike ichabod 18 +mike king 4 +mike king 58 +mike king 83 +mike king 96 +mike king 103 +mike king 118 +mike miller 51 +mike nixon 97 +mike nixon 106 +mike polk 6 +mike polk 65 +mike polk 119 +mike quirinius 22 +mike steinbeck 75 +mike steinbeck 85 +mike steinbeck 101 +mike steinbeck 116 +mike van buren 16 +mike van buren 111 +mike white -1 +mike white 22 +mike white 45 +mike white 61 +mike young 37 +mike young 53 +mike young 72 +mike zipper 27 +mike zipper 76 +mike zipper 106 +nick allen 8 +nick allen 57 +nick brown 114 +nick davidson 84 +nick ellison 10 +nick ellison 107 +nick falkner 83 +nick falkner 86 +nick garcia 53 +nick garcia 69 +nick garcia 108 +nick ichabod 59 +nick ichabod 71 +nick ichabod 84 +nick johnson 47 +nick johnson 88 +nick laertes 17 +nick miller 101 +nick nixon 43 +nick ovid 42 +nick polk 1 +nick quirinius 22 +nick quirinius 36 +nick robinson 48 +nick robinson 54 +nick steinbeck 33 +nick thompson 73 +nick underhill 122 +nick van buren 53 +nick xylophone 80 +nick young 6 +nick young 60 +nick zipper 3 +nick zipper 21 +oscar allen 58 +oscar brown 80 +oscar carson 10 +oscar carson 27 +oscar carson 36 +oscar carson 72 +oscar carson 88 +oscar davidson 14 +oscar ellison 50 +oscar ellison 74 +oscar falkner 96 +oscar garcia 44 +oscar hernandez 1 +oscar hernandez 93 +oscar ichabod 20 +oscar ichabod 28 +oscar ichabod 69 +oscar ichabod 120 +oscar johnson 44 +oscar johnson 53 +oscar king 67 +oscar king 71 +oscar king 81 +oscar laertes 4 +oscar laertes 28 +oscar laertes 53 +oscar laertes 63 +oscar nixon 58 +oscar ovid 3 +oscar ovid 27 +oscar ovid 47 +oscar polk 8 +oscar polk 112 +oscar quirinius NULL +oscar quirinius 0 +oscar quirinius 17 +oscar quirinius 114 +oscar robinson 16 +oscar robinson 42 +oscar robinson 59 +oscar robinson 93 +oscar steinbeck 51 +oscar thompson 44 +oscar thompson 44 +oscar thompson 60 +oscar thompson 66 +oscar underhill 86 +oscar van buren 40 +oscar van buren 51 +oscar van buren 114 +oscar white 2 +oscar white 20 +oscar white 49 +oscar white 58 +oscar xylophone 18 +oscar xylophone 73 +oscar xylophone 74 +oscar zipper 0 +oscar zipper 23 +oscar zipper 95 +priscilla brown 51 +priscilla brown 75 +priscilla brown 97 +priscilla carson 16 +priscilla carson 52 +priscilla carson 124 +priscilla ichabod 117 +priscilla ichabod 122 +priscilla johnson 5 +priscilla johnson 17 +priscilla johnson 62 +priscilla johnson 77 +priscilla johnson 117 +priscilla king 43 +priscilla nixon 61 +priscilla nixon 66 +priscilla ovid 46 +priscilla ovid 118 +priscilla polk 45 +priscilla quirinius 83 +priscilla thompson 82 +priscilla underhill 117 +priscilla underhill 122 +priscilla van buren 0 +priscilla van buren 22 +priscilla van buren 102 +priscilla white 88 +priscilla xylophone 8 +priscilla xylophone 90 +priscilla xylophone 109 +priscilla young 17 +priscilla young 113 +priscilla zipper 27 +priscilla zipper 35 +quinn allen 27 +quinn allen 114 +quinn brown 70 +quinn brown 88 +quinn brown 117 +quinn davidson 93 +quinn davidson 93 +quinn davidson 109 +quinn davidson 121 +quinn ellison 83 +quinn ellison 116 +quinn garcia 78 +quinn garcia 104 +quinn garcia 110 +quinn garcia 120 +quinn ichabod 60 +quinn king 14 +quinn king 46 +quinn laertes -2 +quinn laertes 65 +quinn laertes 95 +quinn nixon 11 +quinn ovid 123 +quinn quirinius 94 +quinn robinson 60 +quinn steinbeck 82 +quinn steinbeck 122 +quinn thompson 41 +quinn thompson 60 +quinn underhill 19 +quinn underhill 28 +quinn underhill 34 +quinn van buren 18 +quinn young 15 +quinn zipper 44 +quinn zipper 103 +rachel allen 76 +rachel allen 122 +rachel brown 23 +rachel brown 56 +rachel brown 71 +rachel brown 101 +rachel brown 108 +rachel carson 27 +rachel carson 74 +rachel davidson 84 +rachel ellison 51 +rachel falkner -2 +rachel falkner 43 +rachel falkner 72 +rachel falkner 104 +rachel johnson 32 +rachel king 84 +rachel king 95 +rachel laertes 37 +rachel laertes 106 +rachel ovid 5 +rachel ovid 31 +rachel polk 79 +rachel quirinius 108 +rachel robinson 24 +rachel robinson 41 +rachel robinson 91 +rachel thompson -3 +rachel thompson -2 +rachel thompson 74 +rachel underhill 11 +rachel white 108 +rachel white 119 +rachel young 77 +rachel zipper 16 +rachel zipper 116 +sarah carson 41 +sarah carson 58 +sarah carson 119 +sarah ellison 14 +sarah falkner 112 +sarah falkner 123 +sarah garcia 72 +sarah garcia 91 +sarah garcia 98 +sarah ichabod 38 +sarah ichabod 80 +sarah johnson 5 +sarah johnson 51 +sarah johnson 69 +sarah johnson 116 +sarah king 13 +sarah king 120 +sarah miller 31 +sarah ovid 122 +sarah robinson 26 +sarah robinson 35 +sarah steinbeck 30 +sarah white 11 +sarah white 32 +sarah xylophone 28 +sarah young 120 +sarah zipper 107 +tom brown 27 +tom brown 89 +tom carson 11 +tom carson 70 +tom carson 123 +tom davidson 72 +tom ellison 28 +tom ellison 118 +tom ellison 120 +tom falkner 11 +tom falkner 35 +tom hernandez -3 +tom hernandez 118 +tom ichabod 19 +tom johnson 42 +tom johnson 82 +tom king 59 +tom laertes 33 +tom laertes 54 +tom miller 9 +tom miller 48 +tom miller 94 +tom nixon 45 +tom ovid 68 +tom polk 70 +tom polk 107 +tom quirinius 10 +tom quirinius 38 +tom robinson 52 +tom robinson 104 +tom robinson 109 +tom robinson 115 +tom steinbeck 113 +tom van buren 5 +tom van buren 48 +tom van buren 63 +tom white 81 +tom young 13 +tom young 99 +tom zipper 31 +ulysses brown 46 +ulysses carson 5 +ulysses carson 26 +ulysses carson 55 +ulysses carson 109 +ulysses davidson 18 +ulysses ellison 61 +ulysses garcia 12 +ulysses hernandez 9 +ulysses hernandez 22 +ulysses hernandez 53 +ulysses ichabod 32 +ulysses ichabod 99 +ulysses johnson 41 +ulysses king 2 +ulysses laertes 40 +ulysses laertes 51 +ulysses laertes 95 +ulysses miller 23 +ulysses miller 85 +ulysses nixon 92 +ulysses ovid 31 +ulysses polk 28 +ulysses polk 74 +ulysses polk 86 +ulysses polk 89 +ulysses quirinius 7 +ulysses robinson 79 +ulysses steinbeck 6 +ulysses steinbeck 45 +ulysses thompson 24 +ulysses underhill 6 +ulysses underhill 27 +ulysses underhill 42 +ulysses underhill 51 +ulysses underhill 93 +ulysses underhill 98 +ulysses underhill 111 +ulysses van buren 58 +ulysses white 67 +ulysses white 109 +ulysses xylophone 47 +ulysses xylophone 105 +ulysses xylophone 123 +ulysses young 61 +ulysses young 86 +ulysses young 89 +victor allen 2 +victor allen 17 +victor brown 0 +victor brown 23 +victor brown 60 +victor brown 64 +victor davidson 42 +victor davidson 89 +victor davidson 123 +victor ellison 35 +victor ellison 84 +victor hernandez 1 +victor hernandez 17 +victor hernandez 91 +victor hernandez 94 +victor hernandez 116 +victor johnson 34 +victor johnson 53 +victor johnson 57 +victor king 59 +victor king 112 +victor laertes 18 +victor laertes 118 +victor miller 79 +victor nixon 50 +victor nixon 104 +victor ovid 120 +victor polk 106 +victor quirinius 77 +victor quirinius 85 +victor robinson 29 +victor robinson 105 +victor steinbeck 20 +victor steinbeck 92 +victor steinbeck 100 +victor thompson 124 +victor van buren 41 +victor van buren 71 +victor white 15 +victor white 49 +victor xylophone -3 +victor xylophone 41 +victor xylophone 43 +victor xylophone 54 +victor xylophone 91 +victor young 24 +victor zipper 3 +wendy allen 25 +wendy allen 38 +wendy allen 95 +wendy brown 92 +wendy brown 119 +wendy ellison 53 +wendy ellison 103 +wendy falkner 23 +wendy falkner 28 +wendy falkner 58 +wendy garcia 3 +wendy garcia 48 +wendy garcia 60 +wendy garcia 99 +wendy hernandez 53 +wendy ichabod 87 +wendy king -2 +wendy king 45 +wendy king 124 +wendy laertes 26 +wendy laertes 51 +wendy laertes 72 +wendy miller 51 +wendy miller 105 +wendy nixon 5 +wendy nixon 25 +wendy ovid 17 +wendy ovid 85 +wendy polk 44 +wendy polk 99 +wendy quirinius 77 +wendy quirinius 88 +wendy robinson -3 +wendy robinson 71 +wendy robinson 97 +wendy steinbeck 37 +wendy thompson 28 +wendy thompson 31 +wendy underhill 58 +wendy underhill 82 +wendy underhill 120 +wendy van buren 27 +wendy van buren 82 +wendy white 63 +wendy xylophone 53 +wendy xylophone 119 +wendy young 66 +wendy young 112 +xavier allen 18 +xavier allen 41 +xavier allen 106 +xavier brown 10 +xavier brown 63 +xavier brown 108 +xavier carson 20 +xavier carson 57 +xavier davidson 21 +xavier davidson 24 +xavier davidson 106 +xavier ellison 0 +xavier ellison 53 +xavier garcia 42 +xavier hernandez 9 +xavier hernandez 80 +xavier hernandez 114 +xavier ichabod 20 +xavier ichabod 58 +xavier johnson 44 +xavier johnson 85 +xavier king 26 +xavier king 107 +xavier laertes 60 +xavier ovid 3 +xavier polk 29 +xavier polk 83 +xavier polk 91 +xavier polk 122 +xavier quirinius 27 +xavier quirinius 35 +xavier quirinius 39 +xavier quirinius 111 +xavier thompson 2 +xavier underhill 102 +xavier white 8 +xavier white 56 +xavier xylophone 24 +xavier zipper 48 +yuri allen 31 +yuri allen 121 +yuri brown 101 +yuri brown 106 +yuri carson 1 +yuri carson 36 +yuri ellison -1 +yuri ellison 43 +yuri falkner 31 +yuri falkner 96 +yuri garcia 49 +yuri hernandez 92 +yuri johnson 1 +yuri johnson 2 +yuri johnson 111 +yuri king 44 +yuri laertes 84 +yuri laertes 115 +yuri nixon 5 +yuri nixon 111 +yuri polk 13 +yuri polk 49 +yuri polk 115 +yuri quirinius 24 +yuri quirinius 28 +yuri quirinius 90 +yuri steinbeck 8 +yuri steinbeck 65 +yuri thompson 42 +yuri underhill 10 +yuri underhill 66 +yuri white 73 +yuri xylophone 63 +zach allen 35 +zach brown 7 +zach brown 15 +zach brown 37 +zach brown 61 +zach brown 94 +zach carson 114 +zach ellison 16 +zach falkner 70 +zach falkner 115 +zach garcia -2 +zach garcia 59 +zach garcia 68 +zach garcia 97 +zach ichabod 14 +zach ichabod 73 +zach king 66 +zach king 70 +zach king 81 +zach miller 4 +zach miller 9 +zach miller 73 +zach ovid 61 +zach ovid 68 +zach ovid 77 +zach ovid 114 +zach quirinius 79 +zach robinson 69 +zach steinbeck 6 +zach steinbeck 122 +zach thompson 75 +zach thompson 95 +zach underhill 123 +zach white 58 +zach xylophone 19 +zach xylophone 85 +zach young 11 +zach zipper 68 +zach zipper 100 +zach zipper 101 diff --git a/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-4-dfd39236756a3951bc1ec354799d69e4 b/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-4-dfd39236756a3951bc1ec354799d69e4 new file mode 100644 index 0000000000000..86ca4e49d21bd --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-4-dfd39236756a3951bc1ec354799d69e4 @@ -0,0 +1,1049 @@ + + + +alice allen +alice allen +alice allen +alice brown +alice carson +alice davidson +alice falkner +alice garcia +alice hernandez +alice hernandez +alice johnson +alice king +alice king +alice king +alice laertes +alice laertes +alice miller +alice nixon +alice nixon +alice nixon +alice ovid +alice polk +alice quirinius +alice quirinius +alice robinson +alice robinson +alice steinbeck +alice steinbeck +alice steinbeck +alice underhill +alice van buren +alice xylophone +alice xylophone +alice xylophone +alice zipper +alice zipper +alice zipper +bob brown +bob brown +bob brown +bob carson +bob davidson +bob davidson +bob davidson +bob ellison +bob ellison +bob ellison +bob ellison +bob falkner +bob garcia +bob garcia +bob garcia +bob garcia +bob garcia +bob hernandez +bob ichabod +bob king +bob king +bob king +bob laertes +bob laertes +bob miller +bob ovid +bob ovid +bob ovid +bob ovid +bob polk +bob quirinius +bob steinbeck +bob van buren +bob white +bob white +bob xylophone +bob xylophone +bob young +bob zipper +bob zipper +bob zipper +calvin allen +calvin brown +calvin brown +calvin brown +calvin carson +calvin davidson +calvin davidson +calvin ellison +calvin falkner +calvin falkner +calvin falkner +calvin falkner +calvin falkner +calvin falkner +calvin garcia +calvin hernandez +calvin johnson +calvin laertes +calvin laertes +calvin nixon +calvin nixon +calvin nixon +calvin ovid +calvin ovid +calvin ovid +calvin ovid +calvin polk +calvin quirinius +calvin quirinius +calvin robinson +calvin steinbeck +calvin steinbeck +calvin steinbeck +calvin thompson +calvin thompson +calvin underhill +calvin van buren +calvin van buren +calvin white +calvin white +calvin xylophone +calvin xylophone +calvin xylophone +calvin young +calvin young +calvin zipper +calvin zipper +david allen +david allen +david brown +david brown +david davidson +david davidson +david davidson +david davidson +david ellison +david ellison +david ellison +david hernandez +david ichabod +david ichabod +david laertes +david nixon +david ovid +david ovid +david quirinius +david quirinius +david quirinius +david robinson +david robinson +david thompson +david underhill +david underhill +david underhill +david van buren +david van buren +david white +david xylophone +david xylophone +david xylophone +david young +david young +ethan allen +ethan brown +ethan brown +ethan brown +ethan brown +ethan brown +ethan brown +ethan carson +ethan ellison +ethan ellison +ethan falkner +ethan falkner +ethan garcia +ethan hernandez +ethan johnson +ethan king +ethan laertes +ethan laertes +ethan laertes +ethan laertes +ethan laertes +ethan laertes +ethan laertes +ethan miller +ethan nixon +ethan ovid +ethan polk +ethan polk +ethan polk +ethan polk +ethan quirinius +ethan quirinius +ethan quirinius +ethan robinson +ethan robinson +ethan underhill +ethan van buren +ethan white +ethan white +ethan xylophone +ethan zipper +ethan zipper +fred davidson +fred davidson +fred davidson +fred ellison +fred ellison +fred ellison +fred falkner +fred falkner +fred falkner +fred hernandez +fred ichabod +fred ichabod +fred johnson +fred king +fred king +fred laertes +fred miller +fred nixon +fred nixon +fred nixon +fred nixon +fred polk +fred polk +fred polk +fred polk +fred quirinius +fred quirinius +fred robinson +fred steinbeck +fred steinbeck +fred steinbeck +fred underhill +fred van buren +fred van buren +fred van buren +fred van buren +fred white +fred young +fred young +fred zipper +gabriella allen +gabriella allen +gabriella brown +gabriella brown +gabriella carson +gabriella davidson +gabriella ellison +gabriella ellison +gabriella falkner +gabriella falkner +gabriella falkner +gabriella garcia +gabriella hernandez +gabriella hernandez +gabriella ichabod +gabriella ichabod +gabriella ichabod +gabriella ichabod +gabriella ichabod +gabriella king +gabriella king +gabriella laertes +gabriella miller +gabriella ovid +gabriella ovid +gabriella polk +gabriella polk +gabriella steinbeck +gabriella steinbeck +gabriella thompson +gabriella thompson +gabriella thompson +gabriella van buren +gabriella van buren +gabriella white +gabriella young +gabriella young +gabriella zipper +gabriella zipper +holly allen +holly brown +holly brown +holly falkner +holly hernandez +holly hernandez +holly hernandez +holly hernandez +holly ichabod +holly ichabod +holly ichabod +holly johnson +holly johnson +holly johnson +holly king +holly king +holly laertes +holly miller +holly nixon +holly nixon +holly polk +holly polk +holly robinson +holly thompson +holly thompson +holly thompson +holly underhill +holly underhill +holly underhill +holly underhill +holly van buren +holly white +holly white +holly xylophone +holly young +holly young +holly zipper +holly zipper +irene allen +irene brown +irene brown +irene brown +irene carson +irene ellison +irene ellison +irene falkner +irene falkner +irene garcia +irene garcia +irene garcia +irene ichabod +irene ichabod +irene johnson +irene laertes +irene laertes +irene laertes +irene miller +irene nixon +irene nixon +irene nixon +irene ovid +irene ovid +irene ovid +irene polk +irene polk +irene polk +irene polk +irene polk +irene quirinius +irene quirinius +irene quirinius +irene robinson +irene steinbeck +irene thompson +irene underhill +irene underhill +irene van buren +irene van buren +irene xylophone +jessica brown +jessica carson +jessica carson +jessica carson +jessica davidson +jessica davidson +jessica davidson +jessica davidson +jessica ellison +jessica ellison +jessica falkner +jessica garcia +jessica garcia +jessica ichabod +jessica johnson +jessica johnson +jessica miller +jessica nixon +jessica nixon +jessica ovid +jessica ovid +jessica polk +jessica quirinius +jessica quirinius +jessica quirinius +jessica quirinius +jessica robinson +jessica thompson +jessica thompson +jessica underhill +jessica underhill +jessica underhill +jessica van buren +jessica white +jessica white +jessica white +jessica white +jessica white +jessica xylophone +jessica young +jessica young +jessica zipper +jessica zipper +jessica zipper +katie allen +katie brown +katie davidson +katie ellison +katie ellison +katie falkner +katie garcia +katie garcia +katie hernandez +katie ichabod +katie ichabod +katie ichabod +katie king +katie king +katie king +katie miller +katie miller +katie nixon +katie ovid +katie polk +katie polk +katie robinson +katie van buren +katie van buren +katie white +katie white +katie xylophone +katie young +katie young +katie young +katie zipper +katie zipper +luke allen +luke allen +luke allen +luke allen +luke allen +luke brown +luke davidson +luke davidson +luke ellison +luke ellison +luke ellison +luke falkner +luke falkner +luke garcia +luke garcia +luke ichabod +luke ichabod +luke johnson +luke johnson +luke johnson +luke laertes +luke laertes +luke laertes +luke laertes +luke laertes +luke miller +luke ovid +luke ovid +luke polk +luke polk +luke quirinius +luke robinson +luke robinson +luke thompson +luke underhill +luke underhill +luke underhill +luke van buren +luke white +luke xylophone +luke zipper +mike allen +mike brown +mike carson +mike carson +mike carson +mike davidson +mike davidson +mike ellison +mike ellison +mike ellison +mike ellison +mike ellison +mike falkner +mike garcia +mike garcia +mike garcia +mike hernandez +mike hernandez +mike ichabod +mike king +mike king +mike king +mike king +mike king +mike king +mike miller +mike nixon +mike nixon +mike polk +mike polk +mike polk +mike quirinius +mike steinbeck +mike steinbeck +mike steinbeck +mike steinbeck +mike van buren +mike van buren +mike white +mike white +mike white +mike white +mike young +mike young +mike young +mike zipper +mike zipper +mike zipper +nick allen +nick allen +nick brown +nick davidson +nick ellison +nick ellison +nick falkner +nick falkner +nick garcia +nick garcia +nick garcia +nick ichabod +nick ichabod +nick ichabod +nick johnson +nick johnson +nick laertes +nick miller +nick nixon +nick ovid +nick polk +nick quirinius +nick quirinius +nick robinson +nick robinson +nick steinbeck +nick thompson +nick underhill +nick van buren +nick xylophone +nick young +nick young +nick zipper +nick zipper +oscar allen +oscar brown +oscar carson +oscar carson +oscar carson +oscar carson +oscar carson +oscar davidson +oscar ellison +oscar ellison +oscar falkner +oscar garcia +oscar hernandez +oscar hernandez +oscar ichabod +oscar ichabod +oscar ichabod +oscar ichabod +oscar johnson +oscar johnson +oscar king +oscar king +oscar king +oscar laertes +oscar laertes +oscar laertes +oscar laertes +oscar nixon +oscar ovid +oscar ovid +oscar ovid +oscar polk +oscar polk +oscar quirinius +oscar quirinius +oscar quirinius +oscar quirinius +oscar robinson +oscar robinson +oscar robinson +oscar robinson +oscar steinbeck +oscar thompson +oscar thompson +oscar thompson +oscar thompson +oscar underhill +oscar van buren +oscar van buren +oscar van buren +oscar white +oscar white +oscar white +oscar white +oscar xylophone +oscar xylophone +oscar xylophone +oscar zipper +oscar zipper +oscar zipper +priscilla brown +priscilla brown +priscilla brown +priscilla carson +priscilla carson +priscilla carson +priscilla ichabod +priscilla ichabod +priscilla johnson +priscilla johnson +priscilla johnson +priscilla johnson +priscilla johnson +priscilla king +priscilla nixon +priscilla nixon +priscilla ovid +priscilla ovid +priscilla polk +priscilla quirinius +priscilla thompson +priscilla underhill +priscilla underhill +priscilla van buren +priscilla van buren +priscilla van buren +priscilla white +priscilla xylophone +priscilla xylophone +priscilla xylophone +priscilla young +priscilla young +priscilla zipper +priscilla zipper +quinn allen +quinn allen +quinn brown +quinn brown +quinn brown +quinn davidson +quinn davidson +quinn davidson +quinn davidson +quinn ellison +quinn ellison +quinn garcia +quinn garcia +quinn garcia +quinn garcia +quinn ichabod +quinn king +quinn king +quinn laertes +quinn laertes +quinn laertes +quinn nixon +quinn ovid +quinn quirinius +quinn robinson +quinn steinbeck +quinn steinbeck +quinn thompson +quinn thompson +quinn underhill +quinn underhill +quinn underhill +quinn van buren +quinn young +quinn zipper +quinn zipper +rachel allen +rachel allen +rachel brown +rachel brown +rachel brown +rachel brown +rachel brown +rachel carson +rachel carson +rachel davidson +rachel ellison +rachel falkner +rachel falkner +rachel falkner +rachel falkner +rachel johnson +rachel king +rachel king +rachel laertes +rachel laertes +rachel ovid +rachel ovid +rachel polk +rachel quirinius +rachel robinson +rachel robinson +rachel robinson +rachel thompson +rachel thompson +rachel thompson +rachel underhill +rachel white +rachel white +rachel young +rachel zipper +rachel zipper +sarah carson +sarah carson +sarah carson +sarah ellison +sarah falkner +sarah falkner +sarah garcia +sarah garcia +sarah garcia +sarah ichabod +sarah ichabod +sarah johnson +sarah johnson +sarah johnson +sarah johnson +sarah king +sarah king +sarah miller +sarah ovid +sarah robinson +sarah robinson +sarah steinbeck +sarah white +sarah white +sarah xylophone +sarah young +sarah zipper +tom brown +tom brown +tom carson +tom carson +tom carson +tom davidson +tom ellison +tom ellison +tom ellison +tom falkner +tom falkner +tom hernandez +tom hernandez +tom ichabod +tom johnson +tom johnson +tom king +tom laertes +tom laertes +tom miller +tom miller +tom miller +tom nixon +tom ovid +tom polk +tom polk +tom quirinius +tom quirinius +tom robinson +tom robinson +tom robinson +tom robinson +tom steinbeck +tom van buren +tom van buren +tom van buren +tom white +tom young +tom young +tom zipper +ulysses brown +ulysses carson +ulysses carson +ulysses carson +ulysses carson +ulysses davidson +ulysses ellison +ulysses garcia +ulysses hernandez +ulysses hernandez +ulysses hernandez +ulysses ichabod +ulysses ichabod +ulysses johnson +ulysses king +ulysses laertes +ulysses laertes +ulysses laertes +ulysses miller +ulysses miller +ulysses nixon +ulysses ovid +ulysses polk +ulysses polk +ulysses polk +ulysses polk +ulysses quirinius +ulysses robinson +ulysses steinbeck +ulysses steinbeck +ulysses thompson +ulysses underhill +ulysses underhill +ulysses underhill +ulysses underhill +ulysses underhill +ulysses underhill +ulysses underhill +ulysses van buren +ulysses white +ulysses white +ulysses xylophone +ulysses xylophone +ulysses xylophone +ulysses young +ulysses young +ulysses young +victor allen +victor allen +victor brown +victor brown +victor brown +victor brown +victor davidson +victor davidson +victor davidson +victor ellison +victor ellison +victor hernandez +victor hernandez +victor hernandez +victor hernandez +victor hernandez +victor johnson +victor johnson +victor johnson +victor king +victor king +victor laertes +victor laertes +victor miller +victor nixon +victor nixon +victor ovid +victor polk +victor quirinius +victor quirinius +victor robinson +victor robinson +victor steinbeck +victor steinbeck +victor steinbeck +victor thompson +victor van buren +victor van buren +victor white +victor white +victor xylophone +victor xylophone +victor xylophone +victor xylophone +victor xylophone +victor young +victor zipper +wendy allen +wendy allen +wendy allen +wendy brown +wendy brown +wendy ellison +wendy ellison +wendy falkner +wendy falkner +wendy falkner +wendy garcia +wendy garcia +wendy garcia +wendy garcia +wendy hernandez +wendy ichabod +wendy king +wendy king +wendy king +wendy laertes +wendy laertes +wendy laertes +wendy miller +wendy miller +wendy nixon +wendy nixon +wendy ovid +wendy ovid +wendy polk +wendy polk +wendy quirinius +wendy quirinius +wendy robinson +wendy robinson +wendy robinson +wendy steinbeck +wendy thompson +wendy thompson +wendy underhill +wendy underhill +wendy underhill +wendy van buren +wendy van buren +wendy white +wendy xylophone +wendy xylophone +wendy young +wendy young +xavier allen +xavier allen +xavier allen +xavier brown +xavier brown +xavier brown +xavier carson +xavier carson +xavier davidson +xavier davidson +xavier davidson +xavier ellison +xavier ellison +xavier garcia +xavier hernandez +xavier hernandez +xavier hernandez +xavier ichabod +xavier ichabod +xavier johnson +xavier johnson +xavier king +xavier king +xavier laertes +xavier ovid +xavier polk +xavier polk +xavier polk +xavier polk +xavier quirinius +xavier quirinius +xavier quirinius +xavier quirinius +xavier thompson +xavier underhill +xavier white +xavier white +xavier xylophone +xavier zipper +yuri allen +yuri allen +yuri brown +yuri brown +yuri carson +yuri carson +yuri ellison +yuri ellison +yuri falkner +yuri falkner +yuri garcia +yuri hernandez +yuri johnson +yuri johnson +yuri johnson +yuri king +yuri laertes +yuri laertes +yuri nixon +yuri nixon +yuri polk +yuri polk +yuri polk +yuri quirinius +yuri quirinius +yuri quirinius +yuri steinbeck +yuri steinbeck +yuri thompson +yuri underhill +yuri underhill +yuri white +yuri xylophone +zach allen +zach brown +zach brown +zach brown +zach brown +zach brown +zach carson +zach ellison +zach falkner +zach falkner +zach garcia +zach garcia +zach garcia +zach garcia +zach ichabod +zach ichabod +zach king +zach king +zach king +zach miller +zach miller +zach miller +zach ovid +zach ovid +zach ovid +zach ovid +zach quirinius +zach robinson +zach steinbeck +zach steinbeck +zach thompson +zach thompson +zach underhill +zach white +zach xylophone +zach xylophone +zach young +zach zipper +zach zipper +zach zipper diff --git a/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-5-8d0ee3e1605f38214bfad28a5ce897cc b/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-5-8d0ee3e1605f38214bfad28a5ce897cc new file mode 100644 index 0000000000000..ddb15e338263f --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-5-8d0ee3e1605f38214bfad28a5ce897cc @@ -0,0 +1 @@ +10 oscar carson 65549 65549 diff --git a/sql/hive/src/test/resources/golden/windowing_ntile.q (deterministic)-0-b7cb25303831392a51cd996e758ac79a b/sql/hive/src/test/resources/golden/windowing_ntile.q (deterministic)-0-b7cb25303831392a51cd996e758ac79a new file mode 100644 index 0000000000000..42e5151fe211b --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_ntile.q (deterministic)-0-b7cb25303831392a51cd996e758ac79a @@ -0,0 +1,1049 @@ +65536 1 +65536 1 +65536 1 +65536 1 +65536 1 +65536 1 +65536 1 +65537 1 +65537 1 +65537 1 +65537 1 +65537 1 +65538 1 +65538 1 +65538 1 +65539 1 +65539 1 +65539 1 +65540 1 +65541 1 +65541 1 +65541 1 +65541 1 +65541 1 +65541 1 +65541 1 +65541 1 +65541 1 +65542 1 +65542 1 +65542 1 +65542 1 +65542 1 +65543 1 +65543 1 +65544 1 +65544 1 +65544 1 +65544 1 +65544 1 +65544 1 +65544 1 +65544 2 +65545 1 +65545 1 +65545 1 +65546 2 +65547 1 +65547 1 +65547 1 +65547 1 +65548 1 +65548 1 +65548 1 +65548 1 +65548 1 +65548 1 +65548 2 +65549 1 +65549 1 +65549 1 +65549 1 +65549 1 +65549 1 +65549 1 +65549 3 +65550 1 +65550 1 +65550 1 +65550 1 +65550 1 +65550 1 +65550 2 +65551 1 +65551 1 +65551 1 +65552 1 +65552 1 +65552 1 +65552 1 +65552 2 +65553 1 +65553 1 +65553 1 +65553 1 +65553 1 +65554 1 +65554 1 +65555 2 +65556 1 +65556 1 +65556 1 +65556 1 +65556 1 +65557 1 +65557 1 +65558 1 +65559 1 +65559 1 +65559 1 +65559 1 +65559 1 +65560 1 +65560 1 +65560 1 +65560 2 +65560 2 +65561 1 +65561 2 +65561 2 +65562 1 +65562 1 +65562 1 +65562 1 +65562 1 +65562 2 +65563 1 +65563 1 +65563 1 +65563 1 +65564 1 +65564 1 +65564 1 +65564 1 +65564 2 +65564 2 +65564 2 +65565 1 +65565 1 +65566 1 +65566 2 +65567 1 +65568 1 +65568 1 +65568 1 +65569 1 +65570 1 +65570 1 +65570 1 +65570 1 +65570 1 +65570 2 +65571 1 +65571 1 +65571 1 +65571 2 +65572 1 +65572 1 +65572 1 +65572 1 +65572 1 +65573 1 +65573 1 +65573 3 +65574 1 +65574 1 +65574 1 +65575 1 +65575 1 +65575 1 +65575 2 +65576 1 +65576 1 +65576 1 +65576 3 +65577 1 +65578 1 +65578 1 +65578 1 +65578 1 +65578 1 +65578 1 +65578 2 +65579 1 +65579 1 +65579 1 +65579 1 +65579 3 +65580 1 +65580 2 +65580 2 +65581 1 +65581 1 +65581 1 +65581 1 +65581 2 +65581 2 +65582 1 +65582 1 +65582 1 +65582 1 +65583 1 +65583 1 +65583 2 +65583 2 +65584 1 +65584 1 +65584 1 +65585 1 +65585 1 +65585 1 +65585 1 +65585 2 +65586 1 +65586 1 +65586 1 +65586 1 +65586 2 +65587 1 +65587 1 +65587 1 +65587 2 +65587 3 +65588 1 +65588 1 +65588 1 +65588 1 +65588 2 +65589 1 +65589 1 +65589 1 +65589 1 +65589 2 +65590 1 +65590 1 +65590 1 +65590 1 +65590 1 +65591 1 +65591 1 +65591 2 +65591 3 +65592 1 +65593 1 +65593 1 +65594 1 +65594 1 +65594 1 +65594 3 +65595 1 +65595 1 +65595 1 +65595 2 +65595 2 +65595 4 +65596 1 +65596 1 +65596 1 +65596 2 +65596 2 +65597 1 +65597 1 +65597 2 +65598 1 +65598 2 +65599 1 +65599 1 +65599 1 +65599 1 +65599 3 +65600 1 +65600 1 +65600 2 +65600 2 +65601 1 +65601 2 +65601 2 +65602 1 +65602 1 +65602 1 +65602 1 +65602 1 +65602 2 +65603 1 +65603 1 +65603 1 +65603 2 +65603 2 +65604 1 +65604 2 +65604 2 +65604 3 +65605 2 +65606 1 +65606 1 +65606 1 +65606 1 +65606 2 +65606 2 +65607 1 +65607 1 +65607 1 +65607 1 +65607 2 +65607 3 +65608 1 +65608 1 +65608 1 +65608 2 +65609 1 +65610 1 +65610 1 +65610 3 +65610 4 +65610 5 +65611 1 +65611 1 +65611 1 +65611 1 +65612 1 +65612 2 +65612 2 +65612 3 +65613 1 +65614 1 +65614 1 +65614 2 +65615 1 +65615 1 +65615 2 +65615 2 +65616 1 +65616 2 +65617 1 +65617 2 +65617 2 +65617 2 +65618 1 +65618 1 +65618 2 +65618 2 +65618 2 +65619 1 +65619 1 +65619 1 +65619 1 +65619 2 +65619 2 +65620 1 +65620 1 +65620 1 +65620 1 +65620 1 +65620 3 +65621 1 +65621 1 +65622 1 +65622 1 +65622 1 +65622 3 +65622 3 +65622 4 +65623 1 +65623 1 +65623 1 +65623 1 +65623 1 +65623 2 +65623 4 +65624 1 +65624 2 +65624 2 +65624 2 +65624 4 +65625 1 +65625 1 +65625 1 +65625 1 +65626 1 +65626 1 +65626 1 +65626 2 +65627 1 +65627 1 +65627 2 +65627 3 +65628 1 +65628 1 +65628 1 +65628 2 +65628 2 +65628 3 +65629 1 +65629 1 +65629 1 +65629 3 +65630 1 +65630 2 +65631 1 +65632 2 +65632 2 +65633 1 +65633 2 +65633 2 +65633 2 +65634 1 +65634 1 +65634 1 +65634 2 +65635 1 +65635 2 +65635 3 +65636 1 +65636 1 +65636 2 +65637 1 +65637 1 +65637 1 +65637 1 +65637 1 +65637 1 +65637 2 +65637 3 +65638 1 +65638 1 +65638 2 +65639 1 +65640 2 +65641 1 +65641 1 +65641 1 +65641 5 +65642 4 +65643 1 +65643 1 +65643 2 +65643 2 +65643 2 +65643 2 +65643 2 +65643 2 +65643 4 +65644 1 +65644 1 +65644 1 +65644 1 +65644 2 +65644 2 +65644 3 +65645 1 +65645 1 +65645 1 +65646 1 +65646 1 +65646 1 +65647 1 +65647 2 +65648 1 +65648 1 +65648 2 +65648 2 +65649 1 +65649 1 +65649 2 +65649 2 +65650 1 +65650 1 +65650 1 +65650 1 +65650 2 +65650 2 +65651 1 +65651 1 +65651 2 +65651 2 +65651 3 +65652 1 +65652 1 +65652 2 +65653 1 +65653 2 +65653 2 +65653 3 +65654 1 +65654 1 +65654 1 +65654 1 +65654 1 +65654 1 +65654 2 +65654 2 +65654 2 +65655 1 +65655 1 +65656 1 +65656 1 +65656 1 +65656 1 +65656 2 +65656 3 +65657 1 +65657 2 +65657 2 +65657 2 +65657 2 +65658 1 +65658 1 +65658 1 +65658 1 +65658 1 +65658 2 +65658 2 +65658 2 +65658 2 +65658 2 +65659 1 +65659 1 +65659 1 +65659 1 +65659 1 +65659 2 +65659 2 +65659 3 +65660 1 +65660 1 +65660 2 +65661 1 +65661 2 +65661 2 +65661 2 +65661 3 +65662 1 +65662 1 +65662 2 +65662 2 +65662 2 +65662 2 +65663 2 +65663 2 +65663 2 +65663 3 +65664 1 +65664 1 +65664 1 +65664 1 +65664 1 +65664 2 +65664 2 +65664 2 +65665 2 +65666 1 +65666 1 +65666 2 +65667 1 +65667 1 +65667 1 +65667 1 +65668 3 +65669 1 +65669 1 +65669 1 +65669 1 +65669 1 +65669 3 +65670 1 +65670 1 +65670 2 +65670 2 +65670 3 +65671 2 +65671 2 +65671 3 +65672 1 +65672 1 +65672 1 +65672 2 +65672 2 +65673 2 +65673 2 +65673 3 +65673 4 +65673 4 +65674 1 +65675 1 +65675 1 +65675 2 +65675 2 +65675 2 +65675 2 +65675 3 +65675 3 +65676 1 +65676 1 +65676 2 +65676 3 +65677 1 +65677 1 +65677 1 +65677 2 +65677 3 +65677 4 +65678 1 +65678 1 +65678 1 +65679 1 +65679 2 +65680 1 +65680 1 +65680 2 +65680 2 +65680 2 +65680 5 +65681 1 +65681 2 +65681 2 +65681 2 +65681 4 +65681 4 +65682 1 +65682 2 +65682 2 +65683 1 +65683 1 +65683 2 +65684 1 +65684 1 +65684 2 +65685 2 +65685 2 +65685 2 +65685 3 +65685 3 +65685 4 +65686 1 +65686 2 +65686 3 +65686 3 +65687 1 +65687 1 +65687 2 +65687 3 +65688 1 +65688 2 +65689 1 +65689 3 +65690 2 +65690 2 +65691 1 +65691 1 +65691 1 +65691 1 +65691 1 +65691 2 +65691 3 +65691 3 +65692 1 +65692 2 +65693 1 +65693 2 +65693 2 +65693 2 +65693 2 +65693 3 +65693 5 +65694 1 +65694 2 +65694 2 +65694 3 +65695 1 +65695 2 +65695 2 +65695 3 +65696 1 +65696 2 +65696 2 +65696 4 +65697 1 +65697 1 +65697 1 +65697 1 +65697 2 +65697 5 +65698 1 +65698 2 +65699 1 +65699 1 +65699 1 +65699 2 +65699 2 +65699 2 +65700 1 +65700 2 +65700 2 +65701 1 +65701 2 +65701 3 +65702 2 +65702 2 +65702 2 +65702 3 +65703 1 +65703 2 +65703 3 +65703 3 +65704 1 +65704 1 +65704 2 +65704 2 +65704 3 +65704 3 +65705 1 +65705 1 +65705 3 +65705 4 +65706 1 +65706 1 +65706 1 +65706 3 +65706 4 +65707 2 +65707 2 +65708 2 +65708 3 +65709 1 +65709 1 +65709 2 +65710 1 +65710 2 +65711 1 +65711 2 +65711 2 +65711 2 +65711 2 +65711 2 +65712 1 +65712 2 +65712 3 +65712 3 +65712 4 +65713 1 +65713 2 +65713 3 +65713 3 +65713 6 +65714 1 +65714 1 +65714 2 +65715 1 +65715 1 +65715 2 +65715 2 +65715 4 +65716 1 +65716 2 +65716 2 +65716 2 +65716 4 +65716 4 +65717 1 +65717 2 +65717 2 +65717 2 +65717 5 +65718 1 +65718 2 +65718 3 +65718 3 +65719 1 +65719 1 +65719 2 +65719 3 +65720 1 +65720 2 +65720 2 +65720 3 +65720 4 +65721 1 +65721 1 +65721 1 +65721 1 +65721 3 +65721 3 +65721 3 +65722 2 +65722 3 +65722 5 +65723 2 +65723 3 +65724 1 +65724 1 +65724 2 +65724 2 +65724 3 +65724 3 +65725 1 +65726 2 +65726 2 +65726 4 +65727 1 +65727 1 +65727 3 +65727 3 +65727 4 +65728 1 +65728 2 +65729 2 +65730 1 +65730 1 +65730 2 +65730 4 +65731 1 +65731 1 +65731 1 +65731 3 +65732 1 +65732 2 +65732 2 +65733 1 +65733 1 +65733 2 +65733 3 +65733 3 +65733 6 +65734 2 +65734 2 +65735 1 +65735 4 +65735 4 +65736 1 +65736 2 +65736 2 +65736 3 +65737 1 +65737 1 +65737 2 +65737 3 +65737 4 +65738 3 +65738 3 +65738 4 +65738 4 +65739 1 +65739 2 +65739 3 +65740 2 +65740 2 +65740 3 +65741 1 +65742 1 +65742 2 +65742 3 +65743 1 +65743 1 +65743 1 +65743 2 +65744 1 +65744 2 +65745 2 +65745 2 +65745 3 +65745 6 +65746 1 +65746 2 +65746 2 +65746 2 +65746 2 +65747 1 +65747 1 +65747 1 +65747 2 +65747 2 +65747 3 +65747 3 +65748 1 +65748 3 +65749 2 +65749 3 +65749 3 +65749 3 +65749 4 +65750 1 +65750 1 +65750 2 +65750 3 +65750 3 +65751 1 +65751 2 +65751 2 +65751 3 +65751 3 +65751 4 +65752 1 +65752 3 +65753 2 +65753 3 +65754 2 +65754 4 +65755 1 +65755 2 +65755 2 +65755 3 +65755 3 +65755 3 +65755 3 +65755 5 +65755 5 +65756 2 +65756 3 +65756 3 +65756 5 +65757 1 +65757 1 +65757 1 +65757 2 +65757 3 +65758 1 +65758 2 +65758 2 +65758 3 +65758 4 +65759 2 +65759 2 +65759 2 +65759 4 +65759 4 +65760 2 +65760 4 +65760 5 +65760 7 +65761 1 +65762 1 +65762 1 +65762 2 +65762 4 +65762 5 +65763 1 +65763 2 +65763 2 +65763 2 +65763 4 +65764 3 +65764 3 +65764 4 +65765 2 +65765 2 +65765 3 +65766 1 +65766 1 +65766 3 +65766 3 +65766 3 +65767 2 +65767 3 +65767 3 +65768 3 +65769 1 +65769 2 +65769 2 +65769 2 +65769 3 +65769 5 +65770 2 +65770 3 +65771 2 +65771 3 +65771 3 +65772 2 +65773 1 +65773 2 +65773 2 +65773 2 +65773 2 +65773 3 +65773 3 +65773 4 +65774 2 +65774 2 +65774 2 +65774 2 +65774 3 +65774 3 +65775 1 +65775 2 +65775 3 +65775 3 +65775 5 +65776 1 +65776 3 +65776 3 +65776 5 +65776 6 +65777 2 +65777 3 +65777 4 +65777 4 +65778 1 +65778 2 +65778 2 +65778 2 +65778 3 +65778 3 +65778 4 +65778 5 +65779 3 +65779 3 +65779 3 +65779 3 +65779 4 +65779 4 +65779 5 +65780 1 +65780 2 +65780 3 +65781 1 +65781 3 +65781 3 +65782 2 +65782 2 +65782 3 +65782 4 +65782 5 +65783 2 +65783 3 +65783 3 +65783 3 +65783 3 +65783 3 +65784 2 +65784 2 +65784 2 +65784 4 +65784 6 +65785 2 +65785 7 +65786 1 +65786 2 +65786 3 +65786 3 +65786 4 +65787 1 +65787 2 +65787 3 +65787 3 +65788 1 +65788 1 +65788 2 +65788 4 +65789 1 +65789 1 +65789 2 +65789 2 +65789 2 +65789 2 +65789 4 +65790 2 +65790 2 +65790 4 +65791 2 +65791 2 diff --git a/sql/hive/src/test/resources/golden/windowing_ntile.q (deterministic)-1-a3d352560ac835993001665db6954965 b/sql/hive/src/test/resources/golden/windowing_ntile.q (deterministic)-1-a3d352560ac835993001665db6954965 new file mode 100644 index 0000000000000..dc72606a83db6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_ntile.q (deterministic)-1-a3d352560ac835993001665db6954965 @@ -0,0 +1,1049 @@ + 1 + 1 + 1 +alice allen 1 +alice allen 1 +alice allen 1 +alice brown 1 +alice carson 1 +alice davidson 1 +alice falkner 1 +alice garcia 1 +alice hernandez 1 +alice hernandez 1 +alice johnson 1 +alice king 1 +alice king 1 +alice king 1 +alice laertes 1 +alice laertes 2 +alice miller 1 +alice nixon 1 +alice nixon 1 +alice nixon 1 +alice ovid 2 +alice polk 1 +alice quirinius 1 +alice quirinius 1 +alice robinson 1 +alice robinson 1 +alice steinbeck 1 +alice steinbeck 1 +alice steinbeck 1 +alice underhill 1 +alice van buren 1 +alice xylophone 1 +alice xylophone 1 +alice xylophone 2 +alice zipper 1 +alice zipper 1 +alice zipper 2 +bob brown 1 +bob brown 1 +bob brown 1 +bob carson 1 +bob davidson 1 +bob davidson 1 +bob davidson 1 +bob ellison 1 +bob ellison 1 +bob ellison 1 +bob ellison 1 +bob falkner 2 +bob garcia 1 +bob garcia 1 +bob garcia 1 +bob garcia 2 +bob garcia 2 +bob hernandez 1 +bob ichabod 1 +bob king 1 +bob king 1 +bob king 2 +bob laertes 1 +bob laertes 1 +bob miller 1 +bob ovid 1 +bob ovid 1 +bob ovid 1 +bob ovid 1 +bob polk 1 +bob quirinius 1 +bob steinbeck 1 +bob van buren 1 +bob white 1 +bob white 1 +bob xylophone 1 +bob xylophone 1 +bob young 1 +bob zipper 1 +bob zipper 2 +bob zipper 2 +calvin allen 3 +calvin brown 1 +calvin brown 1 +calvin brown 1 +calvin carson 2 +calvin davidson 1 +calvin davidson 2 +calvin ellison 1 +calvin falkner 1 +calvin falkner 1 +calvin falkner 1 +calvin falkner 2 +calvin falkner 2 +calvin falkner 2 +calvin garcia 2 +calvin hernandez 3 +calvin johnson 1 +calvin laertes 1 +calvin laertes 1 +calvin nixon 1 +calvin nixon 1 +calvin nixon 1 +calvin ovid 1 +calvin ovid 1 +calvin ovid 2 +calvin ovid 2 +calvin polk 2 +calvin quirinius 1 +calvin quirinius 1 +calvin robinson 1 +calvin steinbeck 1 +calvin steinbeck 1 +calvin steinbeck 2 +calvin thompson 1 +calvin thompson 2 +calvin underhill 1 +calvin van buren 1 +calvin van buren 1 +calvin white 1 +calvin white 2 +calvin xylophone 1 +calvin xylophone 2 +calvin xylophone 2 +calvin young 1 +calvin young 2 +calvin zipper 3 +calvin zipper 4 +david allen 1 +david allen 1 +david brown 2 +david brown 3 +david davidson 1 +david davidson 2 +david davidson 3 +david davidson 3 +david ellison 1 +david ellison 2 +david ellison 3 +david hernandez 1 +david ichabod 1 +david ichabod 3 +david laertes 3 +david nixon 1 +david ovid 1 +david ovid 1 +david quirinius 1 +david quirinius 1 +david quirinius 3 +david robinson 1 +david robinson 4 +david thompson 1 +david underhill 1 +david underhill 2 +david underhill 3 +david van buren 1 +david van buren 2 +david white 1 +david xylophone 1 +david xylophone 1 +david xylophone 2 +david young 1 +david young 1 +ethan allen 1 +ethan brown 1 +ethan brown 1 +ethan brown 1 +ethan brown 1 +ethan brown 2 +ethan brown 2 +ethan carson 1 +ethan ellison 1 +ethan ellison 2 +ethan falkner 1 +ethan falkner 1 +ethan garcia 1 +ethan hernandez 1 +ethan johnson 2 +ethan king 2 +ethan laertes 1 +ethan laertes 1 +ethan laertes 2 +ethan laertes 2 +ethan laertes 2 +ethan laertes 2 +ethan laertes 3 +ethan miller 1 +ethan nixon 2 +ethan ovid 2 +ethan polk 1 +ethan polk 1 +ethan polk 1 +ethan polk 2 +ethan quirinius 1 +ethan quirinius 1 +ethan quirinius 2 +ethan robinson 1 +ethan robinson 2 +ethan underhill 2 +ethan van buren 1 +ethan white 1 +ethan white 2 +ethan xylophone 2 +ethan zipper 1 +ethan zipper 3 +fred davidson 2 +fred davidson 2 +fred davidson 3 +fred ellison 1 +fred ellison 2 +fred ellison 2 +fred falkner 1 +fred falkner 3 +fred falkner 4 +fred hernandez 2 +fred ichabod 2 +fred ichabod 3 +fred johnson 2 +fred king 2 +fred king 2 +fred laertes 2 +fred miller 3 +fred nixon 1 +fred nixon 1 +fred nixon 1 +fred nixon 3 +fred polk 1 +fred polk 1 +fred polk 1 +fred polk 2 +fred quirinius 2 +fred quirinius 3 +fred robinson 2 +fred steinbeck 1 +fred steinbeck 1 +fred steinbeck 1 +fred underhill 1 +fred van buren 2 +fred van buren 3 +fred van buren 3 +fred van buren 4 +fred white 2 +fred young 1 +fred young 2 +fred zipper 3 +gabriella allen 1 +gabriella allen 3 +gabriella brown 1 +gabriella brown 1 +gabriella carson 2 +gabriella davidson 1 +gabriella ellison 1 +gabriella ellison 3 +gabriella falkner 1 +gabriella falkner 1 +gabriella falkner 3 +gabriella garcia 1 +gabriella hernandez 1 +gabriella hernandez 1 +gabriella ichabod 1 +gabriella ichabod 2 +gabriella ichabod 2 +gabriella ichabod 2 +gabriella ichabod 4 +gabriella king 1 +gabriella king 2 +gabriella laertes 1 +gabriella miller 2 +gabriella ovid 1 +gabriella ovid 2 +gabriella polk 1 +gabriella polk 2 +gabriella steinbeck 1 +gabriella steinbeck 1 +gabriella thompson 1 +gabriella thompson 2 +gabriella thompson 3 +gabriella van buren 1 +gabriella van buren 2 +gabriella white 1 +gabriella young 1 +gabriella young 2 +gabriella zipper 1 +gabriella zipper 2 +holly allen 3 +holly brown 2 +holly brown 2 +holly falkner 2 +holly hernandez 2 +holly hernandez 2 +holly hernandez 2 +holly hernandez 3 +holly ichabod 1 +holly ichabod 2 +holly ichabod 2 +holly johnson 1 +holly johnson 3 +holly johnson 4 +holly king 2 +holly king 2 +holly laertes 3 +holly miller 2 +holly nixon 1 +holly nixon 2 +holly polk 1 +holly polk 2 +holly robinson 3 +holly thompson 1 +holly thompson 3 +holly thompson 4 +holly underhill 2 +holly underhill 2 +holly underhill 3 +holly underhill 3 +holly van buren 1 +holly white 4 +holly white 4 +holly xylophone 2 +holly young 1 +holly young 2 +holly zipper 1 +holly zipper 4 +irene allen 3 +irene brown 1 +irene brown 2 +irene brown 3 +irene carson 2 +irene ellison 2 +irene ellison 2 +irene falkner 1 +irene falkner 1 +irene garcia 1 +irene garcia 2 +irene garcia 3 +irene ichabod 1 +irene ichabod 2 +irene johnson 2 +irene laertes 1 +irene laertes 3 +irene laertes 4 +irene miller 1 +irene nixon 1 +irene nixon 3 +irene nixon 3 +irene ovid 2 +irene ovid 2 +irene ovid 2 +irene polk 1 +irene polk 1 +irene polk 2 +irene polk 2 +irene polk 4 +irene quirinius 2 +irene quirinius 3 +irene quirinius 4 +irene robinson 2 +irene steinbeck 1 +irene thompson 1 +irene underhill 2 +irene underhill 3 +irene van buren 2 +irene van buren 3 +irene xylophone 2 +jessica brown 2 +jessica carson 1 +jessica carson 2 +jessica carson 4 +jessica davidson 1 +jessica davidson 2 +jessica davidson 3 +jessica davidson 3 +jessica ellison 1 +jessica ellison 3 +jessica falkner 2 +jessica garcia 1 +jessica garcia 5 +jessica ichabod 2 +jessica johnson 2 +jessica johnson 3 +jessica miller 2 +jessica nixon 2 +jessica nixon 3 +jessica ovid 2 +jessica ovid 3 +jessica polk 5 +jessica quirinius 2 +jessica quirinius 2 +jessica quirinius 3 +jessica quirinius 3 +jessica robinson 1 +jessica thompson 2 +jessica thompson 3 +jessica underhill 2 +jessica underhill 2 +jessica underhill 4 +jessica van buren 2 +jessica white 3 +jessica white 3 +jessica white 3 +jessica white 3 +jessica white 4 +jessica xylophone 4 +jessica young 4 +jessica young 4 +jessica zipper 1 +jessica zipper 2 +jessica zipper 4 +katie allen 2 +katie brown 4 +katie davidson 3 +katie ellison 3 +katie ellison 3 +katie falkner 2 +katie garcia 2 +katie garcia 3 +katie hernandez 2 +katie ichabod 2 +katie ichabod 2 +katie ichabod 2 +katie king 1 +katie king 1 +katie king 2 +katie miller 2 +katie miller 3 +katie nixon 5 +katie ovid 1 +katie polk 2 +katie polk 3 +katie robinson 4 +katie van buren 2 +katie van buren 4 +katie white 1 +katie white 2 +katie xylophone 3 +katie young 2 +katie young 2 +katie young 3 +katie zipper 1 +katie zipper 3 +luke allen 2 +luke allen 2 +luke allen 2 +luke allen 3 +luke allen 3 +luke brown 2 +luke davidson 1 +luke davidson 3 +luke ellison 3 +luke ellison 5 +luke ellison 5 +luke falkner 2 +luke falkner 4 +luke garcia 1 +luke garcia 5 +luke ichabod 3 +luke ichabod 3 +luke johnson 1 +luke johnson 2 +luke johnson 3 +luke laertes 2 +luke laertes 3 +luke laertes 3 +luke laertes 3 +luke laertes 3 +luke miller 2 +luke ovid 1 +luke ovid 2 +luke polk 2 +luke polk 3 +luke quirinius 2 +luke robinson 1 +luke robinson 4 +luke thompson 1 +luke underhill 2 +luke underhill 3 +luke underhill 5 +luke van buren 2 +luke white 3 +luke xylophone 2 +luke zipper 1 +mike allen 4 +mike brown 4 +mike carson 1 +mike carson 2 +mike carson 4 +mike davidson 3 +mike davidson 4 +mike ellison 2 +mike ellison 3 +mike ellison 3 +mike ellison 4 +mike ellison 4 +mike falkner 1 +mike garcia 1 +mike garcia 2 +mike garcia 3 +mike hernandez 2 +mike hernandez 3 +mike ichabod 1 +mike king 1 +mike king 1 +mike king 3 +mike king 3 +mike king 4 +mike king 4 +mike miller 4 +mike nixon 3 +mike nixon 4 +mike polk 4 +mike polk 5 +mike polk 5 +mike quirinius 3 +mike steinbeck 2 +mike steinbeck 3 +mike steinbeck 3 +mike steinbeck 4 +mike van buren 2 +mike van buren 3 +mike white 3 +mike white 4 +mike white 5 +mike white 6 +mike young 2 +mike young 2 +mike young 4 +mike zipper 1 +mike zipper 4 +mike zipper 6 +nick allen 2 +nick allen 2 +nick brown 5 +nick davidson 1 +nick ellison 3 +nick ellison 4 +nick falkner 2 +nick falkner 3 +nick garcia 2 +nick garcia 4 +nick garcia 4 +nick ichabod 3 +nick ichabod 3 +nick ichabod 3 +nick johnson 4 +nick johnson 4 +nick laertes 2 +nick miller 3 +nick nixon 2 +nick ovid 3 +nick polk 4 +nick quirinius 1 +nick quirinius 3 +nick robinson 4 +nick robinson 4 +nick steinbeck 1 +nick thompson 2 +nick underhill 6 +nick van buren 2 +nick xylophone 3 +nick young 3 +nick young 5 +nick zipper 3 +nick zipper 4 +oscar allen 4 +oscar brown 3 +oscar carson 3 +oscar carson 4 +oscar carson 4 +oscar carson 5 +oscar carson 5 +oscar davidson 4 +oscar ellison 1 +oscar ellison 2 +oscar falkner 1 +oscar garcia 5 +oscar hernandez 1 +oscar hernandez 2 +oscar ichabod 2 +oscar ichabod 5 +oscar ichabod 5 +oscar ichabod 6 +oscar johnson 3 +oscar johnson 7 +oscar king 3 +oscar king 4 +oscar king 4 +oscar laertes 1 +oscar laertes 2 +oscar laertes 3 +oscar laertes 3 +oscar nixon 4 +oscar ovid 3 +oscar ovid 3 +oscar ovid 6 +oscar polk 5 +oscar polk 5 +oscar quirinius 2 +oscar quirinius 4 +oscar quirinius 5 +oscar quirinius 6 +oscar robinson 2 +oscar robinson 3 +oscar robinson 5 +oscar robinson 6 +oscar steinbeck 1 +oscar thompson 2 +oscar thompson 3 +oscar thompson 3 +oscar thompson 4 +oscar underhill 2 +oscar van buren 1 +oscar van buren 2 +oscar van buren 5 +oscar white 1 +oscar white 2 +oscar white 5 +oscar white 5 +oscar xylophone 3 +oscar xylophone 3 +oscar xylophone 4 +oscar zipper 2 +oscar zipper 2 +oscar zipper 2 +priscilla brown 2 +priscilla brown 2 +priscilla brown 4 +priscilla carson 3 +priscilla carson 5 +priscilla carson 7 +priscilla ichabod 1 +priscilla ichabod 4 +priscilla johnson 1 +priscilla johnson 2 +priscilla johnson 4 +priscilla johnson 4 +priscilla johnson 6 +priscilla king 3 +priscilla nixon 3 +priscilla nixon 6 +priscilla ovid 3 +priscilla ovid 7 +priscilla polk 4 +priscilla quirinius 3 +priscilla thompson 6 +priscilla underhill 1 +priscilla underhill 4 +priscilla van buren 3 +priscilla van buren 5 +priscilla van buren 5 +priscilla white 1 +priscilla xylophone 2 +priscilla xylophone 3 +priscilla xylophone 3 +priscilla young 5 +priscilla young 8 +priscilla zipper 3 +priscilla zipper 3 +quinn allen 1 +quinn allen 4 +quinn brown 3 +quinn brown 4 +quinn brown 4 +quinn davidson 2 +quinn davidson 4 +quinn davidson 6 +quinn davidson 7 +quinn ellison 3 +quinn ellison 8 +quinn garcia 2 +quinn garcia 3 +quinn garcia 3 +quinn garcia 5 +quinn ichabod 7 +quinn king 1 +quinn king 1 +quinn laertes 2 +quinn laertes 4 +quinn laertes 5 +quinn nixon 4 +quinn ovid 4 +quinn quirinius 5 +quinn robinson 3 +quinn steinbeck 4 +quinn steinbeck 5 +quinn thompson 4 +quinn thompson 6 +quinn underhill 2 +quinn underhill 3 +quinn underhill 7 +quinn van buren 1 +quinn young 2 +quinn zipper 3 +quinn zipper 4 +rachel allen 2 +rachel allen 3 +rachel brown 2 +rachel brown 3 +rachel brown 4 +rachel brown 4 +rachel brown 5 +rachel carson 2 +rachel carson 4 +rachel davidson 7 +rachel ellison 1 +rachel falkner 1 +rachel falkner 3 +rachel falkner 5 +rachel falkner 6 +rachel johnson 9 +rachel king 3 +rachel king 7 +rachel laertes 4 +rachel laertes 6 +rachel ovid 3 +rachel ovid 4 +rachel polk 3 +rachel quirinius 4 +rachel robinson 4 +rachel robinson 4 +rachel robinson 6 +rachel thompson 4 +rachel thompson 5 +rachel thompson 5 +rachel underhill 2 +rachel white 4 +rachel white 5 +rachel young 4 +rachel zipper 1 +rachel zipper 5 +sarah carson 1 +sarah carson 4 +sarah carson 7 +sarah ellison 1 +sarah falkner 4 +sarah falkner 5 +sarah garcia 2 +sarah garcia 2 +sarah garcia 4 +sarah ichabod 3 +sarah ichabod 3 +sarah johnson 3 +sarah johnson 5 +sarah johnson 5 +sarah johnson 6 +sarah king 3 +sarah king 5 +sarah miller 2 +sarah ovid 5 +sarah robinson 5 +sarah robinson 5 +sarah steinbeck 5 +sarah white 4 +sarah white 6 +sarah xylophone 3 +sarah young 5 +sarah zipper 6 +tom brown 2 +tom brown 5 +tom carson 1 +tom carson 3 +tom carson 5 +tom davidson 2 +tom ellison 3 +tom ellison 4 +tom ellison 6 +tom falkner 3 +tom falkner 4 +tom hernandez 1 +tom hernandez 3 +tom ichabod 4 +tom johnson 6 +tom johnson 7 +tom king 3 +tom laertes 3 +tom laertes 3 +tom miller 3 +tom miller 4 +tom miller 5 +tom nixon 4 +tom ovid 3 +tom polk 2 +tom polk 2 +tom quirinius 3 +tom quirinius 5 +tom robinson 2 +tom robinson 3 +tom robinson 3 +tom robinson 5 +tom steinbeck 2 +tom van buren 2 +tom van buren 3 +tom van buren 6 +tom white 5 +tom young 1 +tom young 5 +tom zipper 7 +ulysses brown 2 +ulysses carson 2 +ulysses carson 5 +ulysses carson 6 +ulysses carson 8 +ulysses davidson 3 +ulysses ellison 4 +ulysses garcia 3 +ulysses hernandez 3 +ulysses hernandez 3 +ulysses hernandez 4 +ulysses ichabod 1 +ulysses ichabod 3 +ulysses johnson 5 +ulysses king 2 +ulysses laertes 2 +ulysses laertes 5 +ulysses laertes 6 +ulysses miller 5 +ulysses miller 7 +ulysses nixon 4 +ulysses ovid 4 +ulysses polk 2 +ulysses polk 2 +ulysses polk 3 +ulysses polk 4 +ulysses quirinius 4 +ulysses robinson 1 +ulysses steinbeck 2 +ulysses steinbeck 5 +ulysses thompson 3 +ulysses underhill 2 +ulysses underhill 2 +ulysses underhill 3 +ulysses underhill 4 +ulysses underhill 4 +ulysses underhill 4 +ulysses underhill 5 +ulysses van buren 2 +ulysses white 6 +ulysses white 7 +ulysses xylophone 2 +ulysses xylophone 3 +ulysses xylophone 6 +ulysses young 1 +ulysses young 4 +ulysses young 7 +victor allen 2 +victor allen 3 +victor brown 1 +victor brown 4 +victor brown 5 +victor brown 7 +victor davidson 4 +victor davidson 4 +victor davidson 6 +victor ellison 4 +victor ellison 4 +victor hernandez 2 +victor hernandez 3 +victor hernandez 4 +victor hernandez 4 +victor hernandez 6 +victor johnson 4 +victor johnson 5 +victor johnson 6 +victor king 2 +victor king 6 +victor laertes 3 +victor laertes 5 +victor miller 5 +victor nixon 2 +victor nixon 3 +victor ovid 3 +victor polk 4 +victor quirinius 5 +victor quirinius 5 +victor robinson 5 +victor robinson 5 +victor steinbeck 3 +victor steinbeck 4 +victor steinbeck 5 +victor thompson 6 +victor van buren 5 +victor van buren 6 +victor white 2 +victor white 7 +victor xylophone 4 +victor xylophone 6 +victor xylophone 6 +victor xylophone 8 +victor xylophone 8 +victor young 5 +victor zipper 3 +wendy allen 5 +wendy allen 6 +wendy allen 6 +wendy brown 3 +wendy brown 5 +wendy ellison 3 +wendy ellison 5 +wendy falkner 2 +wendy falkner 4 +wendy falkner 6 +wendy garcia 4 +wendy garcia 4 +wendy garcia 7 +wendy garcia 7 +wendy hernandez 4 +wendy ichabod 4 +wendy king 4 +wendy king 5 +wendy king 7 +wendy laertes 2 +wendy laertes 3 +wendy laertes 5 +wendy miller 4 +wendy miller 4 +wendy nixon 3 +wendy nixon 5 +wendy ovid 5 +wendy ovid 9 +wendy polk 2 +wendy polk 5 +wendy quirinius 3 +wendy quirinius 4 +wendy robinson 5 +wendy robinson 6 +wendy robinson 6 +wendy steinbeck 3 +wendy thompson 4 +wendy thompson 5 +wendy underhill 4 +wendy underhill 5 +wendy underhill 6 +wendy van buren 6 +wendy van buren 6 +wendy white 4 +wendy xylophone 4 +wendy xylophone 6 +wendy young 1 +wendy young 6 +xavier allen 3 +xavier allen 4 +xavier allen 5 +xavier brown 2 +xavier brown 4 +xavier brown 6 +xavier carson 4 +xavier carson 5 +xavier davidson 7 +xavier davidson 8 +xavier davidson 9 +xavier ellison 8 +xavier ellison 8 +xavier garcia 5 +xavier hernandez 5 +xavier hernandez 6 +xavier hernandez 9 +xavier ichabod 3 +xavier ichabod 4 +xavier johnson 2 +xavier johnson 9 +xavier king 3 +xavier king 5 +xavier laertes 4 +xavier ovid 4 +xavier polk 3 +xavier polk 4 +xavier polk 4 +xavier polk 8 +xavier quirinius 3 +xavier quirinius 5 +xavier quirinius 6 +xavier quirinius 6 +xavier thompson 4 +xavier underhill 2 +xavier white 3 +xavier white 3 +xavier xylophone 4 +xavier zipper 3 +yuri allen 2 +yuri allen 3 +yuri brown 2 +yuri brown 3 +yuri carson 5 +yuri carson 6 +yuri ellison 6 +yuri ellison 6 +yuri falkner 6 +yuri falkner 10 +yuri garcia 1 +yuri hernandez 5 +yuri johnson 5 +yuri johnson 5 +yuri johnson 6 +yuri king 7 +yuri laertes 7 +yuri laertes 8 +yuri nixon 3 +yuri nixon 3 +yuri polk 3 +yuri polk 5 +yuri polk 6 +yuri quirinius 3 +yuri quirinius 4 +yuri quirinius 7 +yuri steinbeck 1 +yuri steinbeck 2 +yuri thompson 3 +yuri underhill 4 +yuri underhill 4 +yuri white 8 +yuri xylophone 3 +zach allen 4 +zach brown 5 +zach brown 5 +zach brown 5 +zach brown 5 +zach brown 7 +zach carson 5 +zach ellison 2 +zach falkner 4 +zach falkner 6 +zach garcia 4 +zach garcia 5 +zach garcia 7 +zach garcia 8 +zach ichabod 4 +zach ichabod 4 +zach king 4 +zach king 5 +zach king 8 +zach miller 1 +zach miller 3 +zach miller 4 +zach ovid 4 +zach ovid 5 +zach ovid 5 +zach ovid 7 +zach quirinius 8 +zach robinson 5 +zach steinbeck 4 +zach steinbeck 6 +zach thompson 3 +zach thompson 4 +zach underhill 3 +zach white 6 +zach xylophone 3 +zach xylophone 5 +zach young 4 +zach zipper 4 +zach zipper 4 +zach zipper 5 diff --git a/sql/hive/src/test/resources/golden/windowing_ntile.q (deterministic)-2-fafa16c0f7697ca28aeb6f2698799562 b/sql/hive/src/test/resources/golden/windowing_ntile.q (deterministic)-2-fafa16c0f7697ca28aeb6f2698799562 new file mode 100644 index 0000000000000..76cbeb254c0e9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_ntile.q (deterministic)-2-fafa16c0f7697ca28aeb6f2698799562 @@ -0,0 +1,1049 @@ +0.08 1 +0.1 1 +0.13 1 +0.15 1 +0.27 1 +0.28 1 +0.43 1 +0.52 1 +0.56 1 +0.6 1 +0.61 1 +0.79 1 +0.84 1 +0.98 1 +1.02 1 +1.08 1 +1.08 1 +1.12 1 +1.21 1 +1.25 1 +1.27 1 +1.29 1 +1.31 1 +1.58 1 +1.87 1 +1.91 1 +1.92 1 +2.07 1 +2.18 1 +2.2 1 +2.35 1 +2.6 1 +2.79 1 +2.92 1 +2.96 1 +2.96 1 +2.97 1 +3.0 1 +3.21 1 +3.28 1 +3.33 1 +3.61 1 +3.62 1 +3.82 1 +3.86 1 +3.96 1 +3.97 1 +4.17 1 +4.32 1 +4.35 1 +4.41 1 +4.46 1 +4.47 1 +4.57 1 +4.59 1 +4.71 1 +4.72 1 +4.79 1 +4.8 1 +4.92 1 +5.08 1 +5.24 1 +5.28 1 +5.4 1 +5.44 1 +5.45 1 +5.51 1 +5.54 1 +5.62 1 +5.67 1 +5.85 1 +5.88 1 +6.29 1 +6.55 1 +6.57 1 +6.63 1 +6.67 1 +6.72 1 +6.74 1 +6.84 1 +6.87 1 +7.05 1 +7.06 1 +7.11 1 +7.54 1 +7.56 1 +7.79 1 +7.82 1 +7.96 1 +7.96 1 +7.98 1 +8.07 1 +8.07 1 +8.32 1 +8.37 1 +8.42 1 +8.45 1 +8.45 1 +8.45 1 +8.45 1 +8.57 1 +8.61 1 +8.67 2 +8.71 1 +8.79 1 +8.91 1 +9.04 1 +9.13 1 +9.19 1 +9.22 1 +9.25 1 +9.26 1 +9.35 1 +9.48 1 +9.56 1 +9.57 1 +9.57 1 +9.68 1 +9.7 1 +9.71 1 +9.74 1 +9.8 1 +9.81 1 +9.93 1 +10.09 1 +10.09 1 +10.13 1 +10.16 1 +10.17 1 +10.19 1 +10.2 1 +10.22 1 +10.25 1 +10.26 1 +10.29 1 +10.6 1 +10.66 1 +10.67 1 +10.73 1 +11.15 1 +11.18 1 +11.19 1 +11.22 1 +11.34 1 +11.55 1 +11.57 1 +11.68 1 +11.82 1 +11.89 1 +11.91 1 +12.02 1 +12.16 1 +12.19 1 +12.32 1 +12.42 1 +12.44 1 +12.45 1 +12.46 1 +12.5 1 +12.54 1 +12.85 1 +12.9 1 +13.01 1 +13.1 1 +13.15 1 +13.35 1 +13.87 1 +13.89 1 +13.94 1 +13.99 1 +14.13 1 +14.21 2 +14.3 1 +14.44 1 +14.84 2 +14.92 1 +14.92 1 +14.93 1 +15.1 1 +15.15 1 +15.18 1 +15.22 1 +15.26 2 +15.3 1 +15.37 1 +15.45 1 +15.63 2 +15.75 1 +15.81 1 +15.86 1 +15.9 1 +15.92 1 +16.08 1 +16.09 1 +16.24 1 +16.25 1 +16.48 1 +16.69 1 +16.99 1 +16.99 1 +17.16 1 +17.37 1 +17.74 1 +17.79 1 +17.87 1 +18.2 1 +18.5 1 +18.56 1 +18.63 1 +18.63 1 +18.86 1 +18.89 1 +18.93 1 +19.0 1 +19.03 1 +19.06 1 +19.06 1 +19.13 1 +19.14 1 +19.28 1 +19.69 1 +20.07 1 +20.38 1 +20.64 1 +20.67 1 +20.79 1 +20.81 1 +20.82 1 +20.82 1 +21.18 1 +21.19 1 +21.23 1 +21.28 1 +21.32 1 +21.45 1 +21.49 1 +21.61 1 +21.7 1 +21.8 1 +21.94 1 +22.01 1 +22.08 1 +22.12 1 +22.12 1 +22.25 1 +22.27 1 +22.36 1 +22.68 1 +22.78 1 +22.85 1 +22.85 1 +22.94 1 +23.07 1 +23.13 1 +23.17 1 +23.19 1 +23.44 1 +23.45 1 +23.6 1 +23.77 1 +23.96 1 +24.02 1 +24.28 1 +24.49 1 +24.52 1 +24.73 1 +24.79 1 +24.8 1 +24.83 1 +24.86 1 +25.11 1 +25.28 1 +25.37 1 +25.42 1 +25.55 1 +25.67 1 +25.88 1 +26.08 1 +26.39 1 +26.43 1 +26.47 1 +26.49 1 +26.49 1 +26.64 1 +26.71 2 +26.73 1 +26.76 1 +27.07 1 +27.12 1 +27.3 1 +27.31 1 +27.63 1 +27.66 1 +27.72 2 +27.87 1 +28.11 1 +28.31 1 +28.45 1 +28.5 1 +28.56 1 +28.69 1 +28.71 1 +28.79 1 +28.89 1 +28.95 1 +29.02 1 +29.24 1 +29.36 1 +29.4 1 +29.41 2 +29.54 1 +29.59 1 +29.78 1 +30.25 2 +30.36 1 +30.37 1 +30.61 1 +30.62 1 +30.63 1 +30.65 1 +30.71 1 +30.81 1 +31.01 1 +31.15 1 +31.4 1 +31.61 1 +31.67 1 +31.77 1 +31.86 1 +31.91 1 +32.01 1 +32.18 1 +32.2 1 +32.23 1 +32.25 1 +32.37 1 +32.41 1 +32.47 1 +32.52 1 +32.75 1 +32.89 2 +32.92 1 +33.36 1 +33.52 1 +33.55 1 +33.58 1 +33.67 1 +33.76 1 +33.83 1 +33.85 2 +33.87 1 +34.03 1 +34.21 1 +34.35 1 +34.41 2 +34.58 1 +34.73 1 +34.97 1 +35.0 2 +35.08 1 +35.13 1 +35.17 1 +35.17 1 +35.56 1 +35.62 1 +35.65 1 +35.68 1 +35.72 1 +35.8 1 +35.89 1 +36.22 1 +36.26 1 +36.58 1 +36.7 1 +36.79 1 +36.89 1 +36.95 1 +37.07 2 +37.1 1 +37.14 1 +37.14 1 +37.24 1 +37.59 1 +37.6 1 +37.72 2 +37.78 1 +37.8 1 +37.85 1 +37.9 1 +38.05 1 +38.05 1 +38.3 2 +38.33 1 +38.57 1 +38.62 1 +38.79 1 +38.85 1 +38.88 1 +38.94 1 +39.01 1 +39.03 1 +39.18 1 +39.21 1 +39.34 1 +39.69 1 +39.81 1 +39.82 1 +39.83 1 +39.87 1 +39.9 1 +39.98 1 +40.0 1 +40.04 1 +40.17 1 +40.24 1 +40.42 1 +40.44 1 +40.78 1 +40.8 1 +40.98 1 +41.2 2 +41.29 1 +41.29 1 +41.31 2 +41.34 1 +41.34 1 +41.36 1 +41.44 1 +41.45 2 +41.62 1 +41.68 1 +41.71 1 +41.81 1 +41.85 1 +41.87 1 +41.89 2 +42.24 1 +42.31 1 +42.42 2 +42.48 1 +42.51 1 +42.55 1 +42.56 1 +42.67 2 +42.76 1 +42.85 1 +43.01 1 +43.02 1 +43.13 1 +43.16 1 +43.17 1 +43.19 1 +43.31 1 +43.37 1 +43.57 1 +43.71 1 +43.73 1 +43.92 1 +44.1 1 +44.22 1 +44.27 1 +44.43 1 +44.57 1 +45.06 2 +45.1 1 +45.19 1 +45.19 1 +45.24 1 +45.34 1 +45.35 1 +45.42 1 +45.45 1 +45.56 1 +45.59 1 +45.68 1 +45.92 1 +45.99 2 +46.02 1 +46.09 1 +46.1 1 +46.15 1 +46.18 1 +46.21 1 +46.27 1 +46.43 1 +46.45 1 +46.62 1 +46.8 1 +46.86 1 +46.87 1 +46.88 1 +46.97 2 +47.08 2 +47.27 1 +47.32 2 +47.57 1 +47.59 1 +47.69 1 +47.88 1 +48.01 1 +48.08 1 +48.11 1 +48.15 1 +48.22 1 +48.23 1 +48.25 1 +48.28 1 +48.37 1 +48.45 1 +48.45 1 +48.52 1 +48.59 1 +49.12 1 +49.28 2 +49.44 1 +49.68 1 +49.77 2 +49.78 1 +50.02 3 +50.08 1 +50.09 1 +50.26 1 +50.28 1 +50.31 1 +50.32 1 +50.4 1 +50.41 1 +50.66 1 +50.7 1 +50.83 2 +50.92 2 +50.96 1 +51.25 1 +51.29 1 +51.29 2 +51.72 1 +51.79 2 +51.84 1 +51.85 1 +52.17 1 +52.23 1 +52.44 1 +52.5 1 +52.53 2 +52.72 1 +52.73 1 +52.85 2 +52.87 1 +53.02 1 +53.06 1 +53.18 2 +53.27 1 +53.59 2 +53.78 1 +53.93 1 +53.94 1 +54.1 2 +54.31 1 +54.34 1 +54.43 1 +54.44 1 +54.47 1 +54.73 1 +54.75 1 +54.83 1 +54.99 1 +55.1 1 +55.18 1 +55.2 1 +55.39 1 +55.51 1 +55.63 1 +55.99 1 +56.04 1 +56.07 1 +56.1 1 +56.15 1 +56.33 1 +56.62 1 +56.68 2 +56.81 1 +57.08 1 +57.11 1 +57.12 2 +57.23 1 +57.25 1 +57.29 1 +57.35 1 +57.37 1 +57.46 1 +57.64 1 +57.67 1 +57.89 2 +57.93 1 +58.0 1 +58.08 2 +58.09 2 +58.13 1 +58.43 1 +58.52 1 +58.66 1 +58.67 1 +58.75 1 +58.86 2 +59.07 1 +59.16 1 +59.21 1 +59.34 1 +59.43 1 +59.45 1 +59.45 2 +59.5 1 +59.55 1 +59.61 1 +59.62 1 +59.68 1 +59.68 1 +59.7 1 +59.71 1 +59.83 1 +59.87 1 +59.99 1 +60.02 1 +60.06 1 +60.12 1 +60.13 1 +60.22 2 +60.26 1 +60.26 1 +60.53 1 +60.6 2 +60.71 1 +60.85 1 +61.21 1 +61.7 1 +61.86 1 +61.88 1 +61.92 2 +61.94 1 +62.14 1 +62.2 1 +62.23 1 +62.3 1 +62.39 1 +62.52 1 +62.72 1 +62.74 1 +62.85 2 +62.9 3 +62.92 1 +63.12 1 +63.33 1 +63.35 2 +63.42 1 +63.51 1 +63.9 1 +64.0 1 +64.22 1 +64.25 1 +64.3 1 +64.36 2 +64.46 1 +64.65 2 +64.67 1 +64.77 1 +64.87 1 +64.95 1 +65.02 1 +65.02 1 +65.38 1 +65.43 1 +65.43 1 +65.44 2 +65.55 1 +65.62 1 +65.7 1 +65.72 1 +66.17 1 +66.17 2 +66.36 1 +66.51 1 +66.61 1 +66.61 1 +66.67 1 +66.89 1 +67.12 1 +67.18 1 +67.26 1 +67.38 1 +67.45 1 +67.48 1 +67.59 1 +67.94 1 +67.98 1 +68.01 2 +68.04 1 +68.22 1 +68.25 1 +68.25 1 +68.32 1 +68.41 1 +68.5 1 +68.81 1 +68.85 2 +68.89 1 +68.95 1 +68.96 1 +69.32 2 +69.53 1 +69.74 3 +69.8 2 +69.88 1 +69.96 1 +69.97 1 +70.0 2 +70.04 1 +70.06 1 +70.24 1 +70.35 1 +70.38 1 +70.39 1 +70.52 1 +70.53 1 +70.56 1 +70.85 1 +70.89 1 +70.93 1 +71.01 1 +71.07 2 +71.13 1 +71.19 1 +71.26 1 +71.31 1 +71.32 1 +71.35 1 +71.5 1 +71.54 1 +71.55 3 +71.68 1 +71.68 2 +71.78 2 +71.8 1 +71.89 2 +72.04 1 +72.18 1 +72.51 1 +72.53 2 +72.56 1 +72.62 1 +72.79 1 +72.98 1 +73.18 1 +73.32 1 +73.48 2 +73.63 2 +73.65 1 +73.68 1 +73.88 1 +73.93 1 +74.0 1 +74.02 1 +74.15 1 +74.19 1 +74.19 1 +74.3 1 +74.42 1 +74.45 1 +74.52 1 +74.53 1 +74.59 1 +74.62 1 +74.72 1 +74.78 1 +75.03 2 +75.1 3 +75.19 1 +75.29 1 +75.35 2 +75.42 2 +75.66 2 +75.73 2 +75.83 1 +75.88 1 +76.05 3 +76.1 1 +76.28 1 +76.28 1 +76.33 1 +76.52 1 +76.69 1 +76.7 1 +76.71 1 +76.72 2 +76.72 2 +76.74 1 +76.92 1 +76.93 1 +77.02 3 +77.1 1 +77.36 2 +77.42 1 +77.57 2 +77.66 1 +77.81 1 +77.84 2 +77.89 1 +77.97 1 +78.21 2 +78.26 1 +78.28 1 +78.3 2 +78.31 1 +78.62 1 +78.64 1 +78.73 1 +78.89 1 +78.98 2 +79.12 1 +79.19 1 +79.21 1 +79.38 1 +79.42 1 +79.48 1 +79.48 2 +79.49 2 +79.54 1 +79.55 2 +79.75 1 +79.83 1 +79.96 1 +79.97 2 +79.99 1 +80.23 2 +80.3 1 +80.3 2 +80.46 1 +80.52 1 +80.58 1 +80.6 1 +80.71 1 +80.74 1 +80.84 2 +80.92 1 +80.96 1 +80.97 1 +80.99 1 +81.17 1 +81.32 1 +81.32 1 +81.47 1 +81.58 1 +81.64 1 +81.66 1 +82.24 1 +82.3 1 +82.34 2 +82.41 1 +82.52 1 +82.55 1 +82.56 1 +82.72 1 +82.97 1 +83.08 1 +83.27 1 +83.33 1 +83.4 1 +83.54 2 +83.57 1 +83.58 1 +83.87 1 +83.92 1 +83.93 1 +84.03 1 +84.23 2 +84.31 1 +84.38 1 +84.4 1 +84.69 1 +84.72 1 +84.83 1 +85.0 1 +85.03 2 +85.1 1 +85.14 1 +85.23 1 +85.49 1 +85.49 2 +85.51 2 +85.74 1 +85.76 1 +85.87 1 +85.9 1 +86.0 1 +86.22 1 +86.23 1 +86.63 1 +86.69 1 +86.92 2 +86.93 1 +86.93 1 +87.14 2 +87.22 1 +87.4 1 +87.48 1 +87.57 1 +87.61 1 +87.67 1 +87.83 2 +87.94 1 +87.99 1 +88.02 1 +88.05 1 +88.07 2 +88.17 1 +88.22 1 +88.36 1 +88.47 1 +88.48 1 +88.55 1 +88.77 1 +88.78 1 +88.8 1 +88.91 2 +89.01 2 +89.03 1 +89.1 3 +89.15 2 +89.28 1 +89.38 1 +89.53 1 +89.55 1 +89.55 1 +89.55 1 +89.8 1 +89.81 1 +89.93 1 +90.05 1 +90.05 1 +90.07 1 +90.12 1 +90.2 1 +90.25 1 +90.28 2 +90.35 1 +90.38 1 +90.51 1 +90.56 2 +90.69 1 +90.69 1 +90.73 1 +90.77 1 +91.05 1 +91.16 1 +91.42 1 +91.48 1 +91.53 1 +91.61 1 +91.63 1 +91.78 1 +91.88 1 +91.97 1 +92.05 2 +92.11 2 +92.33 2 +92.37 1 +92.4 2 +92.55 1 +92.61 1 +92.82 1 +92.96 2 +92.98 1 +93.03 1 +93.09 1 +93.11 1 +93.61 1 +93.64 1 +93.73 1 +94.08 1 +94.15 1 +94.25 1 +94.27 1 +94.31 1 +94.33 1 +94.34 1 +94.38 1 +94.43 1 +94.54 1 +94.66 1 +94.68 1 +94.68 1 +94.72 1 +95.07 1 +95.11 1 +95.28 1 +95.33 1 +95.34 1 +95.38 2 +95.48 1 +95.53 1 +95.53 1 +95.81 1 +95.81 2 +95.84 1 +96.09 1 +96.23 1 +96.25 1 +96.29 1 +96.38 1 +96.62 1 +96.68 1 +96.73 1 +96.78 1 +96.91 2 +96.94 1 +97.09 1 +97.24 1 +97.26 1 +97.39 1 +97.46 1 +97.51 2 +97.56 1 +97.57 1 +97.65 2 +97.68 1 +97.71 4 +97.81 1 +97.83 1 +97.87 1 +98.18 2 +98.22 1 +98.23 1 +98.31 1 +98.48 1 +98.51 1 +98.57 1 +98.72 1 +98.96 1 +99.13 1 +99.15 1 +99.21 1 +99.24 1 +99.29 1 +99.36 1 +99.62 1 +99.65 1 +99.67 1 +99.68 1 +99.91 1 +99.92 1 diff --git a/sql/hive/src/test/resources/golden/windowing_ntile.q (deterministic)-3-bda0e7c77d6f4712a03389cb5032bc6d b/sql/hive/src/test/resources/golden/windowing_ntile.q (deterministic)-3-bda0e7c77d6f4712a03389cb5032bc6d new file mode 100644 index 0000000000000..a9ec53c0cb21c --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_ntile.q (deterministic)-3-bda0e7c77d6f4712a03389cb5032bc6d @@ -0,0 +1,1049 @@ +0.02 1 +0.21 1 +0.27 1 +0.37 1 +0.37 1 +0.47 1 +0.48 1 +0.52 1 +0.6 1 +0.63 1 +0.63 1 +0.66 1 +0.73 1 +0.74 1 +0.74 1 +0.78 1 +0.8 1 +0.86 1 +0.86 1 +0.98 1 +1.17 1 +1.29 1 +1.3 1 +1.31 1 +1.37 1 +1.42 1 +1.45 1 +1.5 1 +1.53 1 +1.61 1 +1.62 1 +1.69 1 +1.71 1 +1.79 1 +1.98 1 +2.07 1 +2.09 1 +2.13 1 +2.16 1 +2.26 1 +2.29 1 +2.34 1 +2.34 1 +2.36 1 +2.43 1 +2.43 1 +2.52 1 +2.53 1 +2.53 1 +2.61 1 +2.63 1 +2.74 1 +2.75 1 +2.75 1 +2.79 1 +2.8 1 +2.82 1 +2.89 1 +2.89 1 +2.89 1 +2.9 1 +2.92 1 +3.03 1 +3.07 1 +3.07 1 +3.12 1 +3.15 1 +3.17 1 +3.27 1 +3.27 1 +3.29 1 +3.4 1 +3.4 1 +3.4 1 +3.4 1 +3.42 1 +3.66 1 +3.67 1 +3.69 1 +3.71 1 +3.78 1 +3.83 1 +3.86 1 +3.92 1 +3.98 1 +3.99 1 +4.04 1 +4.16 1 +4.25 1 +4.27 1 +4.44 1 +4.48 1 +4.53 1 +4.57 1 +4.58 1 +4.62 1 +4.81 1 +4.82 1 +4.83 1 +4.92 1 +4.95 1 +4.96 1 +4.97 1 +4.98 1 +5.09 1 +5.11 1 +5.19 1 +5.23 1 +5.3 1 +5.31 1 +5.31 1 +5.35 1 +5.42 1 +5.51 1 +5.55 1 +5.58 1 +5.74 1 +5.82 1 +5.84 1 +5.93 1 +5.93 1 +5.96 1 +6.06 1 +6.06 1 +6.21 1 +6.28 1 +6.46 1 +6.52 2 +6.54 1 +6.56 1 +6.57 1 +6.58 1 +6.61 1 +6.62 1 +6.76 1 +6.81 1 +6.81 1 +6.96 1 +6.98 1 +7.02 1 +7.03 1 +7.14 1 +7.18 1 +7.24 1 +7.24 1 +7.31 1 +7.36 1 +7.37 1 +7.45 1 +7.53 1 +7.62 1 +7.66 1 +7.71 1 +7.71 1 +7.8 1 +7.92 1 +8.05 1 +8.09 1 +8.21 1 +8.33 1 +8.33 1 +8.49 1 +8.49 1 +8.52 1 +8.56 1 +8.61 1 +8.62 2 +8.72 1 +8.76 1 +8.79 1 +8.82 1 +8.84 1 +8.95 1 +8.98 1 +9.14 1 +9.19 1 +9.21 1 +9.22 1 +9.26 1 +9.27 1 +9.39 2 +9.4 1 +9.42 1 +9.5 1 +9.51 1 +9.56 1 +9.6 1 +9.61 1 +9.62 1 +9.64 1 +9.81 1 +9.87 1 +9.88 1 +9.93 1 +9.94 1 +9.96 1 +9.99 1 +10.15 1 +10.21 1 +10.22 1 +10.23 1 +10.24 1 +10.36 1 +10.38 1 +10.38 1 +10.41 1 +10.47 1 +10.49 1 +10.49 1 +10.51 1 +10.52 1 +10.7 1 +10.71 1 +10.85 1 +10.99 1 +11.02 1 +11.12 1 +11.12 1 +11.16 1 +11.2 1 +11.26 1 +11.27 1 +11.35 1 +11.35 1 +11.4 1 +11.43 1 +11.44 1 +11.44 1 +11.46 1 +11.48 1 +11.5 1 +11.54 1 +11.63 1 +11.66 1 +11.69 1 +11.83 1 +11.9 1 +11.91 1 +11.96 1 +12.02 1 +12.13 1 +12.14 1 +12.15 1 +12.3 1 +12.3 1 +12.3 2 +12.34 1 +12.35 1 +12.43 1 +12.43 1 +12.64 1 +12.66 1 +12.7 1 +12.72 1 +12.73 1 +12.74 2 +12.82 1 +12.85 1 +13.02 1 +13.04 1 +13.08 1 +13.14 1 +13.2 1 +13.2 1 +13.22 1 +13.23 1 +13.3 1 +13.3 1 +13.44 1 +13.44 1 +13.44 1 +13.49 1 +13.6 1 +13.66 1 +13.71 1 +13.72 1 +13.8 1 +13.83 1 +13.84 1 +13.88 1 +13.95 1 +14.07 1 +14.16 1 +14.17 1 +14.22 1 +14.24 1 +14.26 1 +14.29 1 +14.33 1 +14.39 1 +14.44 1 +14.51 1 +14.51 1 +14.52 1 +14.62 1 +14.69 1 +14.72 1 +14.75 1 +14.83 1 +14.83 1 +14.84 1 +14.9 1 +14.91 1 +14.92 1 +14.99 1 +15.0 1 +15.01 1 +15.09 1 +15.09 1 +15.09 1 +15.1 1 +15.12 1 +15.13 1 +15.16 1 +15.18 1 +15.22 1 +15.27 1 +15.28 1 +15.32 1 +15.38 1 +15.46 1 +15.46 1 +15.51 1 +15.54 1 +15.87 1 +15.94 1 +15.97 1 +15.98 1 +16.04 2 +16.1 1 +16.12 1 +16.13 1 +16.15 1 +16.29 1 +16.35 1 +16.36 1 +16.38 1 +16.4 1 +16.42 1 +16.47 1 +16.49 1 +16.54 1 +16.61 1 +16.66 1 +16.79 1 +16.79 1 +16.82 1 +16.87 1 +16.87 1 +16.9 1 +16.9 1 +16.91 1 +16.92 1 +17.03 1 +17.03 2 +17.08 1 +17.15 1 +17.19 1 +17.29 1 +17.33 1 +17.44 1 +17.46 1 +17.47 1 +17.51 1 +17.52 2 +17.55 1 +17.59 1 +17.63 1 +17.69 1 +17.76 1 +17.86 1 +17.89 1 +17.99 1 +18.09 1 +18.19 1 +18.2 1 +18.28 1 +18.29 1 +18.31 1 +18.34 1 +18.35 1 +18.36 1 +18.38 1 +18.38 1 +18.41 1 +18.47 1 +18.48 1 +18.79 1 +18.82 1 +18.83 1 +18.86 1 +18.86 1 +19.03 1 +19.12 1 +19.15 1 +19.2 1 +19.31 1 +19.32 1 +19.41 1 +19.47 1 +19.47 2 +19.56 1 +19.59 1 +19.63 1 +19.65 1 +19.72 1 +19.72 1 +19.79 1 +19.79 1 +19.85 1 +19.87 1 +19.9 1 +19.92 1 +19.93 1 +19.98 1 +20.02 1 +20.02 1 +20.17 1 +20.19 1 +20.22 1 +20.3 1 +20.3 1 +20.34 1 +20.39 1 +20.42 1 +20.42 1 +20.44 1 +20.55 1 +20.55 1 +20.56 1 +20.56 1 +20.58 1 +20.58 1 +20.64 1 +20.65 1 +20.75 1 +20.76 1 +20.76 1 +20.8 1 +20.82 1 +20.91 1 +20.93 1 +20.94 1 +20.94 1 +20.97 1 +21.0 1 +21.01 1 +21.01 1 +21.02 1 +21.02 1 +21.11 1 +21.11 1 +21.14 1 +21.16 1 +21.26 1 +21.27 1 +21.3 1 +21.3 1 +21.33 1 +21.33 1 +21.38 1 +21.42 1 +21.52 1 +21.53 1 +21.57 1 +21.66 1 +21.67 1 +21.69 1 +21.77 1 +21.81 1 +21.86 1 +21.91 1 +21.95 1 +22.15 1 +22.19 1 +22.19 1 +22.22 1 +22.27 1 +22.27 1 +22.33 1 +22.48 1 +22.6 1 +22.61 1 +22.64 1 +22.68 1 +22.73 1 +22.75 1 +22.94 1 +22.95 1 +23.03 1 +23.07 1 +23.15 1 +23.15 1 +23.18 1 +23.18 1 +23.25 1 +23.27 1 +23.3 1 +23.31 1 +23.45 1 +23.48 1 +23.53 1 +23.55 1 +23.59 1 +23.61 1 +23.63 1 +23.73 1 +23.77 1 +23.78 2 +23.88 1 +23.91 1 +24.03 1 +24.03 1 +24.13 1 +24.17 1 +24.18 1 +24.25 1 +24.35 1 +24.35 2 +24.42 1 +24.46 1 +24.53 2 +24.59 1 +24.61 1 +24.61 1 +24.84 1 +24.85 1 +24.86 1 +24.95 1 +25.01 1 +25.02 1 +25.03 1 +25.08 1 +25.11 1 +25.17 1 +25.32 2 +25.36 1 +25.36 1 +25.43 1 +25.49 1 +25.51 1 +25.51 1 +25.58 1 +25.59 1 +25.63 1 +25.71 1 +25.75 1 +25.8 1 +25.92 1 +25.92 1 +25.95 1 +25.97 2 +26.0 1 +26.17 2 +26.21 1 +26.22 1 +26.24 1 +26.28 1 +26.44 1 +26.55 1 +26.55 1 +26.65 1 +26.67 1 +26.71 1 +26.71 1 +26.73 1 +26.74 1 +26.79 1 +26.84 1 +26.87 1 +27.02 1 +27.14 1 +27.2 1 +27.27 1 +27.29 1 +27.36 1 +27.39 1 +27.4 1 +27.42 1 +27.46 1 +27.54 1 +27.54 1 +27.61 1 +27.62 1 +27.89 1 +28.02 1 +28.1 1 +28.13 1 +28.14 1 +28.15 1 +28.17 1 +28.19 1 +28.29 1 +28.36 1 +28.4 1 +28.42 2 +28.44 1 +28.52 1 +28.52 1 +28.61 1 +28.64 1 +28.68 1 +28.69 1 +28.69 1 +28.71 1 +28.71 2 +28.77 1 +28.77 1 +28.85 1 +28.86 1 +28.91 1 +28.96 1 +28.96 1 +28.98 1 +29.0 1 +29.11 1 +29.19 1 +29.22 1 +29.24 1 +29.25 1 +29.36 1 +29.41 1 +29.46 1 +29.49 1 +29.52 2 +29.62 1 +29.63 1 +29.66 1 +29.73 1 +29.76 1 +29.78 1 +29.88 1 +29.96 1 +30.04 1 +30.04 1 +30.09 1 +30.12 1 +30.13 1 +30.16 1 +30.17 1 +30.21 1 +30.22 1 +30.28 1 +30.35 1 +30.37 2 +30.41 1 +30.41 1 +30.49 1 +30.55 1 +30.55 1 +30.58 1 +30.61 1 +30.62 1 +30.66 1 +30.67 1 +30.78 1 +30.78 1 +30.87 1 +30.89 1 +30.9 1 +30.92 1 +30.98 2 +30.99 1 +31.01 1 +31.16 1 +31.23 1 +31.3 1 +31.33 1 +31.36 1 +31.45 1 +31.46 1 +31.5 1 +31.61 1 +31.63 1 +31.64 1 +31.66 1 +31.68 1 +31.74 1 +31.75 1 +31.76 1 +31.84 1 +32.04 1 +32.06 1 +32.13 1 +32.17 1 +32.2 1 +32.25 1 +32.25 1 +32.31 1 +32.33 1 +32.39 1 +32.56 1 +32.56 1 +32.61 1 +32.74 1 +32.85 1 +32.89 1 +32.98 1 +33.0 1 +33.01 1 +33.02 1 +33.02 1 +33.02 1 +33.11 1 +33.12 1 +33.18 1 +33.18 1 +33.19 1 +33.24 1 +33.3 1 +33.36 1 +33.36 1 +33.38 1 +33.49 1 +33.52 2 +33.6 2 +33.64 1 +33.64 1 +33.66 1 +33.67 1 +33.72 1 +33.76 1 +33.9 1 +34.05 1 +34.06 1 +34.11 1 +34.14 1 +34.15 1 +34.17 1 +34.2 1 +34.41 2 +34.48 2 +34.49 1 +34.52 1 +34.53 1 +34.54 1 +34.62 1 +34.68 1 +34.68 1 +34.72 1 +34.81 1 +34.83 1 +34.84 1 +34.9 1 +34.95 1 +34.95 1 +34.97 1 +34.97 1 +34.98 1 +35.01 1 +35.02 1 +35.1 1 +35.15 1 +35.16 1 +35.23 1 +35.24 1 +35.36 1 +35.36 1 +35.49 1 +35.62 1 +35.68 1 +35.72 1 +35.84 1 +35.85 1 +35.9 1 +36.05 2 +36.09 1 +36.11 1 +36.12 1 +36.13 1 +36.22 1 +36.56 1 +36.57 1 +36.57 1 +36.62 1 +36.7 1 +36.72 1 +36.73 1 +36.84 1 +36.86 1 +36.93 1 +36.93 1 +37.02 1 +37.08 1 +37.12 1 +37.23 2 +37.32 1 +37.34 1 +37.37 1 +37.76 1 +37.77 1 +37.8 1 +37.93 1 +37.94 1 +37.96 1 +38.0 1 +38.04 1 +38.04 1 +38.04 1 +38.05 1 +38.07 1 +38.14 1 +38.22 1 +38.28 1 +38.37 2 +38.39 1 +38.43 1 +38.53 1 +38.6 1 +38.62 1 +38.66 1 +38.67 1 +38.73 1 +38.74 1 +38.92 1 +38.94 1 +39.01 1 +39.04 1 +39.05 1 +39.11 1 +39.27 1 +39.29 1 +39.41 1 +39.46 1 +39.49 1 +39.55 1 +39.57 1 +39.6 1 +39.63 1 +39.73 1 +39.74 1 +39.78 2 +39.8 1 +39.84 1 +39.84 1 +39.85 1 +39.92 1 +40.01 1 +40.04 1 +40.15 1 +40.21 2 +40.22 1 +40.24 1 +40.26 1 +40.27 1 +40.39 2 +40.43 1 +40.46 1 +40.5 1 +40.59 1 +40.6 1 +40.63 1 +40.76 1 +40.79 1 +40.84 1 +40.89 1 +40.91 1 +40.94 1 +40.96 1 +41.02 1 +41.08 1 +41.2 1 +41.24 1 +41.33 1 +41.34 1 +41.34 2 +41.36 1 +41.36 1 +41.37 1 +41.54 1 +41.56 1 +41.69 1 +41.73 1 +41.75 1 +41.75 1 +41.83 1 +41.86 1 +41.9 1 +42.0 2 +42.02 1 +42.03 1 +42.04 1 +42.22 1 +42.3 1 +42.37 1 +42.41 2 +42.47 1 +42.55 1 +42.56 1 +42.57 1 +42.76 1 +42.89 1 +42.96 1 +42.96 1 +43.0 1 +43.04 1 +43.04 3 +43.18 1 +43.23 1 +43.3 1 +43.34 1 +43.34 2 +43.4 1 +43.42 1 +43.58 1 +43.64 1 +43.67 1 +43.76 2 +43.84 1 +43.85 1 +43.92 1 +43.95 1 +43.95 1 +43.96 1 +43.96 1 +44.04 1 +44.11 1 +44.12 1 +44.12 1 +44.22 3 +44.24 1 +44.27 1 +44.32 1 +44.36 1 +44.4 1 +44.57 1 +44.6 1 +44.63 1 +44.66 1 +44.73 1 +44.75 1 +44.8 1 +44.83 1 +44.9 1 +44.92 1 +44.93 1 +45.0 1 +45.02 1 +45.06 1 +45.06 1 +45.09 1 +45.1 1 +45.1 1 +45.11 1 +45.12 2 +45.14 1 +45.14 2 +45.28 2 +45.29 1 +45.46 1 +45.49 1 +45.53 1 +45.53 2 +45.54 1 +45.69 1 +45.71 1 +45.78 1 +45.81 2 +45.86 1 +45.9 1 +45.94 1 +46.03 1 +46.03 1 +46.09 1 +46.18 1 +46.19 1 +46.28 1 +46.3 1 +46.31 1 +46.33 1 +46.36 1 +46.39 1 +46.52 1 +46.53 1 +46.54 1 +46.57 1 +46.59 1 +46.67 1 +46.69 1 +46.73 1 +46.73 1 +46.73 2 +46.74 1 +46.81 1 +46.87 1 +46.88 1 +46.9 1 +46.93 1 +46.98 1 +47.0 1 +47.03 1 +47.03 1 +47.06 1 +47.15 1 +47.22 1 +47.3 1 +47.31 2 +47.37 2 +47.4 1 +47.46 1 +47.49 1 +47.55 1 +47.6 1 +47.66 1 +47.68 1 +47.71 1 +47.72 1 +47.82 1 +47.86 2 +47.91 1 +47.91 2 +47.95 1 +47.98 1 +48.0 1 +48.08 1 +48.5 1 +48.52 1 +48.71 1 +48.78 1 +48.8 1 +48.85 1 +48.89 1 +48.96 1 +48.98 1 +49.04 1 +49.05 1 +49.16 1 +49.21 1 +49.32 1 +49.34 1 +49.34 1 +49.38 1 +49.44 2 +49.45 1 +49.45 1 +49.46 1 +49.46 1 +49.52 1 +49.56 1 +49.59 1 +49.63 1 +49.67 1 +49.69 1 +49.71 2 +49.72 1 +49.73 1 +49.79 1 +49.84 1 +49.85 2 diff --git a/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 1-0-2e0cbc2d7c5f16657edacd9e7209e6e7 b/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 1-0-2e0cbc2d7c5f16657edacd9e7209e6e7 new file mode 100644 index 0000000000000..612bdf44c0cdf --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 1-0-2e0cbc2d7c5f16657edacd9e7209e6e7 @@ -0,0 +1,1049 @@ + 1 + 1 + 1 +alice allen 1 +alice allen 1 +alice allen 1 +alice brown 1 +alice carson 1 +alice davidson 1 +alice falkner 1 +alice garcia 2 +alice hernandez 1 +alice hernandez 1 +alice johnson 1 +alice king 1 +alice king 1 +alice king 1 +alice laertes 1 +alice laertes 1 +alice miller 1 +alice nixon 1 +alice nixon 1 +alice nixon 1 +alice ovid 1 +alice polk 1 +alice quirinius 1 +alice quirinius 1 +alice robinson 1 +alice robinson 1 +alice steinbeck 1 +alice steinbeck 1 +alice steinbeck 1 +alice underhill 1 +alice van buren 1 +alice xylophone 1 +alice xylophone 1 +alice xylophone 1 +alice zipper 1 +alice zipper 1 +alice zipper 1 +bob brown 1 +bob brown 1 +bob brown 2 +bob carson 1 +bob davidson 1 +bob davidson 1 +bob davidson 1 +bob ellison 1 +bob ellison 1 +bob ellison 1 +bob ellison 2 +bob falkner 1 +bob garcia 1 +bob garcia 1 +bob garcia 1 +bob garcia 1 +bob garcia 2 +bob hernandez 1 +bob ichabod 1 +bob king 1 +bob king 1 +bob king 1 +bob laertes 1 +bob laertes 1 +bob miller 1 +bob ovid 1 +bob ovid 1 +bob ovid 1 +bob ovid 1 +bob polk 1 +bob quirinius 1 +bob steinbeck 1 +bob van buren 1 +bob white 1 +bob white 1 +bob xylophone 1 +bob xylophone 1 +bob young 1 +bob zipper 1 +bob zipper 1 +bob zipper 1 +calvin allen 1 +calvin brown 1 +calvin brown 1 +calvin brown 1 +calvin carson 1 +calvin davidson 1 +calvin davidson 1 +calvin ellison 1 +calvin falkner 1 +calvin falkner 1 +calvin falkner 1 +calvin falkner 1 +calvin falkner 1 +calvin falkner 1 +calvin garcia 1 +calvin hernandez 1 +calvin johnson 2 +calvin laertes 1 +calvin laertes 1 +calvin nixon 1 +calvin nixon 1 +calvin nixon 1 +calvin ovid 1 +calvin ovid 1 +calvin ovid 1 +calvin ovid 1 +calvin polk 1 +calvin quirinius 1 +calvin quirinius 1 +calvin robinson 1 +calvin steinbeck 1 +calvin steinbeck 1 +calvin steinbeck 1 +calvin thompson 1 +calvin thompson 1 +calvin underhill 1 +calvin van buren 1 +calvin van buren 1 +calvin white 1 +calvin white 1 +calvin xylophone 1 +calvin xylophone 1 +calvin xylophone 1 +calvin young 1 +calvin young 1 +calvin zipper 1 +calvin zipper 1 +david allen 1 +david allen 1 +david brown 1 +david brown 1 +david davidson 1 +david davidson 1 +david davidson 1 +david davidson 2 +david ellison 1 +david ellison 1 +david ellison 1 +david hernandez 1 +david ichabod 1 +david ichabod 1 +david laertes 1 +david nixon 1 +david ovid 1 +david ovid 1 +david quirinius 1 +david quirinius 1 +david quirinius 1 +david robinson 1 +david robinson 1 +david thompson 1 +david underhill 1 +david underhill 1 +david underhill 1 +david van buren 1 +david van buren 1 +david white 2 +david xylophone 1 +david xylophone 1 +david xylophone 1 +david young 1 +david young 1 +ethan allen 1 +ethan brown 1 +ethan brown 1 +ethan brown 1 +ethan brown 1 +ethan brown 1 +ethan brown 1 +ethan carson 1 +ethan ellison 1 +ethan ellison 1 +ethan falkner 1 +ethan falkner 1 +ethan garcia 1 +ethan hernandez 1 +ethan johnson 1 +ethan king 1 +ethan laertes 1 +ethan laertes 1 +ethan laertes 1 +ethan laertes 1 +ethan laertes 1 +ethan laertes 1 +ethan laertes 1 +ethan miller 1 +ethan nixon 1 +ethan ovid 1 +ethan polk 1 +ethan polk 1 +ethan polk 1 +ethan polk 1 +ethan quirinius 1 +ethan quirinius 1 +ethan quirinius 1 +ethan robinson 1 +ethan robinson 1 +ethan underhill 1 +ethan van buren 1 +ethan white 1 +ethan white 1 +ethan xylophone 1 +ethan zipper 1 +ethan zipper 1 +fred davidson 1 +fred davidson 1 +fred davidson 1 +fred ellison 1 +fred ellison 1 +fred ellison 1 +fred falkner 1 +fred falkner 1 +fred falkner 1 +fred hernandez 1 +fred ichabod 1 +fred ichabod 2 +fred johnson 1 +fred king 1 +fred king 1 +fred laertes 1 +fred miller 1 +fred nixon 1 +fred nixon 1 +fred nixon 1 +fred nixon 2 +fred polk 1 +fred polk 1 +fred polk 1 +fred polk 1 +fred quirinius 1 +fred quirinius 1 +fred robinson 1 +fred steinbeck 1 +fred steinbeck 1 +fred steinbeck 1 +fred underhill 1 +fred van buren 1 +fred van buren 1 +fred van buren 1 +fred van buren 1 +fred white 1 +fred young 1 +fred young 1 +fred zipper 1 +gabriella allen 1 +gabriella allen 1 +gabriella brown 1 +gabriella brown 1 +gabriella carson 1 +gabriella davidson 1 +gabriella ellison 1 +gabriella ellison 1 +gabriella falkner 1 +gabriella falkner 1 +gabriella falkner 1 +gabriella garcia 1 +gabriella hernandez 1 +gabriella hernandez 1 +gabriella ichabod 1 +gabriella ichabod 1 +gabriella ichabod 1 +gabriella ichabod 1 +gabriella ichabod 1 +gabriella king 1 +gabriella king 1 +gabriella laertes 1 +gabriella miller 1 +gabriella ovid 1 +gabriella ovid 1 +gabriella polk 1 +gabriella polk 1 +gabriella steinbeck 1 +gabriella steinbeck 1 +gabriella thompson 1 +gabriella thompson 1 +gabriella thompson 1 +gabriella van buren 1 +gabriella van buren 1 +gabriella white 1 +gabriella young 1 +gabriella young 1 +gabriella zipper 1 +gabriella zipper 1 +holly allen 1 +holly brown 1 +holly brown 1 +holly falkner 1 +holly hernandez 1 +holly hernandez 1 +holly hernandez 1 +holly hernandez 2 +holly ichabod 1 +holly ichabod 1 +holly ichabod 1 +holly johnson 1 +holly johnson 1 +holly johnson 1 +holly king 1 +holly king 1 +holly laertes 1 +holly miller 1 +holly nixon 1 +holly nixon 1 +holly polk 1 +holly polk 1 +holly robinson 1 +holly thompson 1 +holly thompson 1 +holly thompson 1 +holly underhill 1 +holly underhill 1 +holly underhill 1 +holly underhill 1 +holly van buren 1 +holly white 1 +holly white 2 +holly xylophone 1 +holly young 1 +holly young 1 +holly zipper 1 +holly zipper 1 +irene allen 1 +irene brown 1 +irene brown 1 +irene brown 1 +irene carson 1 +irene ellison 1 +irene ellison 1 +irene falkner 1 +irene falkner 1 +irene garcia 1 +irene garcia 1 +irene garcia 1 +irene ichabod 1 +irene ichabod 1 +irene johnson 1 +irene laertes 1 +irene laertes 1 +irene laertes 1 +irene miller 1 +irene nixon 1 +irene nixon 1 +irene nixon 1 +irene ovid 1 +irene ovid 1 +irene ovid 1 +irene polk 1 +irene polk 1 +irene polk 1 +irene polk 1 +irene polk 1 +irene quirinius 1 +irene quirinius 1 +irene quirinius 1 +irene robinson 1 +irene steinbeck 1 +irene thompson 1 +irene underhill 1 +irene underhill 1 +irene van buren 1 +irene van buren 1 +irene xylophone 2 +jessica brown 2 +jessica carson 1 +jessica carson 1 +jessica carson 1 +jessica davidson 1 +jessica davidson 1 +jessica davidson 1 +jessica davidson 1 +jessica ellison 1 +jessica ellison 1 +jessica falkner 1 +jessica garcia 1 +jessica garcia 1 +jessica ichabod 1 +jessica johnson 1 +jessica johnson 1 +jessica miller 1 +jessica nixon 1 +jessica nixon 1 +jessica ovid 1 +jessica ovid 2 +jessica polk 1 +jessica quirinius 1 +jessica quirinius 1 +jessica quirinius 1 +jessica quirinius 1 +jessica robinson 1 +jessica thompson 1 +jessica thompson 3 +jessica underhill 1 +jessica underhill 1 +jessica underhill 1 +jessica van buren 1 +jessica white 1 +jessica white 1 +jessica white 1 +jessica white 1 +jessica white 1 +jessica xylophone 1 +jessica young 1 +jessica young 1 +jessica zipper 1 +jessica zipper 1 +jessica zipper 1 +katie allen 1 +katie brown 1 +katie davidson 1 +katie ellison 1 +katie ellison 1 +katie falkner 1 +katie garcia 1 +katie garcia 1 +katie hernandez 1 +katie ichabod 1 +katie ichabod 1 +katie ichabod 1 +katie king 1 +katie king 1 +katie king 1 +katie miller 1 +katie miller 1 +katie nixon 1 +katie ovid 1 +katie polk 1 +katie polk 1 +katie robinson 1 +katie van buren 1 +katie van buren 1 +katie white 1 +katie white 1 +katie xylophone 1 +katie young 1 +katie young 1 +katie young 1 +katie zipper 1 +katie zipper 1 +luke allen 1 +luke allen 1 +luke allen 1 +luke allen 1 +luke allen 2 +luke brown 1 +luke davidson 1 +luke davidson 1 +luke ellison 1 +luke ellison 1 +luke ellison 1 +luke falkner 1 +luke falkner 1 +luke garcia 1 +luke garcia 1 +luke ichabod 1 +luke ichabod 1 +luke johnson 1 +luke johnson 1 +luke johnson 1 +luke laertes 1 +luke laertes 1 +luke laertes 1 +luke laertes 1 +luke laertes 1 +luke miller 1 +luke ovid 1 +luke ovid 1 +luke polk 1 +luke polk 1 +luke quirinius 1 +luke robinson 1 +luke robinson 1 +luke thompson 1 +luke underhill 1 +luke underhill 1 +luke underhill 2 +luke van buren 1 +luke white 1 +luke xylophone 1 +luke zipper 1 +mike allen 1 +mike brown 1 +mike carson 1 +mike carson 1 +mike carson 1 +mike davidson 1 +mike davidson 1 +mike ellison 1 +mike ellison 1 +mike ellison 1 +mike ellison 1 +mike ellison 1 +mike falkner 1 +mike garcia 1 +mike garcia 1 +mike garcia 1 +mike hernandez 1 +mike hernandez 2 +mike ichabod 1 +mike king 1 +mike king 1 +mike king 1 +mike king 1 +mike king 1 +mike king 2 +mike miller 1 +mike nixon 1 +mike nixon 1 +mike polk 1 +mike polk 1 +mike polk 1 +mike quirinius 1 +mike steinbeck 1 +mike steinbeck 1 +mike steinbeck 1 +mike steinbeck 1 +mike van buren 1 +mike van buren 1 +mike white 1 +mike white 1 +mike white 1 +mike white 2 +mike young 1 +mike young 1 +mike young 1 +mike zipper 1 +mike zipper 1 +mike zipper 1 +nick allen 1 +nick allen 1 +nick brown 1 +nick davidson 1 +nick ellison 1 +nick ellison 1 +nick falkner 1 +nick falkner 1 +nick garcia 1 +nick garcia 1 +nick garcia 1 +nick ichabod 1 +nick ichabod 1 +nick ichabod 1 +nick johnson 1 +nick johnson 1 +nick laertes 1 +nick miller 1 +nick nixon 1 +nick ovid 1 +nick polk 1 +nick quirinius 1 +nick quirinius 1 +nick robinson 1 +nick robinson 1 +nick steinbeck 1 +nick thompson 1 +nick underhill 1 +nick van buren 1 +nick xylophone 1 +nick young 1 +nick young 1 +nick zipper 1 +nick zipper 1 +oscar allen 2 +oscar brown 1 +oscar carson 1 +oscar carson 1 +oscar carson 1 +oscar carson 1 +oscar carson 1 +oscar davidson 1 +oscar ellison 1 +oscar ellison 1 +oscar falkner 1 +oscar garcia 1 +oscar hernandez 1 +oscar hernandez 1 +oscar ichabod 1 +oscar ichabod 1 +oscar ichabod 1 +oscar ichabod 1 +oscar johnson 1 +oscar johnson 1 +oscar king 1 +oscar king 1 +oscar king 2 +oscar laertes 1 +oscar laertes 1 +oscar laertes 1 +oscar laertes 1 +oscar nixon 1 +oscar ovid 1 +oscar ovid 1 +oscar ovid 2 +oscar polk 1 +oscar polk 1 +oscar quirinius 1 +oscar quirinius 1 +oscar quirinius 1 +oscar quirinius 1 +oscar robinson 1 +oscar robinson 1 +oscar robinson 1 +oscar robinson 1 +oscar steinbeck 1 +oscar thompson 1 +oscar thompson 1 +oscar thompson 1 +oscar thompson 2 +oscar underhill 1 +oscar van buren 1 +oscar van buren 1 +oscar van buren 1 +oscar white 1 +oscar white 1 +oscar white 1 +oscar white 1 +oscar xylophone 1 +oscar xylophone 1 +oscar xylophone 1 +oscar zipper 1 +oscar zipper 1 +oscar zipper 1 +priscilla brown 1 +priscilla brown 1 +priscilla brown 1 +priscilla carson 1 +priscilla carson 1 +priscilla carson 1 +priscilla ichabod 1 +priscilla ichabod 1 +priscilla johnson 1 +priscilla johnson 1 +priscilla johnson 1 +priscilla johnson 1 +priscilla johnson 1 +priscilla king 1 +priscilla nixon 1 +priscilla nixon 2 +priscilla ovid 1 +priscilla ovid 1 +priscilla polk 1 +priscilla quirinius 1 +priscilla thompson 1 +priscilla underhill 1 +priscilla underhill 1 +priscilla van buren 1 +priscilla van buren 1 +priscilla van buren 1 +priscilla white 1 +priscilla xylophone 1 +priscilla xylophone 1 +priscilla xylophone 1 +priscilla young 1 +priscilla young 1 +priscilla zipper 1 +priscilla zipper 1 +quinn allen 1 +quinn allen 1 +quinn brown 1 +quinn brown 1 +quinn brown 1 +quinn davidson 1 +quinn davidson 1 +quinn davidson 1 +quinn davidson 1 +quinn ellison 1 +quinn ellison 1 +quinn garcia 1 +quinn garcia 1 +quinn garcia 1 +quinn garcia 1 +quinn ichabod 1 +quinn king 1 +quinn king 1 +quinn laertes 1 +quinn laertes 1 +quinn laertes 1 +quinn nixon 1 +quinn ovid 1 +quinn quirinius 1 +quinn robinson 1 +quinn steinbeck 1 +quinn steinbeck 4 +quinn thompson 1 +quinn thompson 2 +quinn underhill 1 +quinn underhill 1 +quinn underhill 2 +quinn van buren 1 +quinn young 1 +quinn zipper 1 +quinn zipper 1 +rachel allen 1 +rachel allen 1 +rachel brown 1 +rachel brown 1 +rachel brown 1 +rachel brown 1 +rachel brown 2 +rachel carson 1 +rachel carson 1 +rachel davidson 1 +rachel ellison 1 +rachel falkner 1 +rachel falkner 1 +rachel falkner 1 +rachel falkner 1 +rachel johnson 1 +rachel king 1 +rachel king 1 +rachel laertes 1 +rachel laertes 1 +rachel ovid 1 +rachel ovid 1 +rachel polk 1 +rachel quirinius 1 +rachel robinson 1 +rachel robinson 1 +rachel robinson 1 +rachel thompson 1 +rachel thompson 1 +rachel thompson 1 +rachel underhill 1 +rachel white 1 +rachel white 1 +rachel young 1 +rachel zipper 1 +rachel zipper 1 +sarah carson 1 +sarah carson 1 +sarah carson 1 +sarah ellison 2 +sarah falkner 1 +sarah falkner 1 +sarah garcia 1 +sarah garcia 1 +sarah garcia 2 +sarah ichabod 1 +sarah ichabod 1 +sarah johnson 1 +sarah johnson 1 +sarah johnson 1 +sarah johnson 1 +sarah king 1 +sarah king 1 +sarah miller 1 +sarah ovid 1 +sarah robinson 1 +sarah robinson 1 +sarah steinbeck 1 +sarah white 1 +sarah white 1 +sarah xylophone 1 +sarah young 1 +sarah zipper 1 +tom brown 1 +tom brown 1 +tom carson 1 +tom carson 1 +tom carson 1 +tom davidson 1 +tom ellison 1 +tom ellison 1 +tom ellison 1 +tom falkner 1 +tom falkner 1 +tom hernandez 1 +tom hernandez 1 +tom ichabod 1 +tom johnson 1 +tom johnson 2 +tom king 1 +tom laertes 1 +tom laertes 1 +tom miller 1 +tom miller 1 +tom miller 2 +tom nixon 1 +tom ovid 1 +tom polk 1 +tom polk 1 +tom quirinius 1 +tom quirinius 1 +tom robinson 1 +tom robinson 1 +tom robinson 2 +tom robinson 2 +tom steinbeck 2 +tom van buren 1 +tom van buren 1 +tom van buren 1 +tom white 1 +tom young 1 +tom young 2 +tom zipper 1 +ulysses brown 1 +ulysses carson 1 +ulysses carson 1 +ulysses carson 1 +ulysses carson 1 +ulysses davidson 1 +ulysses ellison 1 +ulysses garcia 1 +ulysses hernandez 1 +ulysses hernandez 1 +ulysses hernandez 2 +ulysses ichabod 1 +ulysses ichabod 1 +ulysses johnson 1 +ulysses king 1 +ulysses laertes 1 +ulysses laertes 1 +ulysses laertes 1 +ulysses miller 1 +ulysses miller 1 +ulysses nixon 1 +ulysses ovid 1 +ulysses polk 1 +ulysses polk 1 +ulysses polk 1 +ulysses polk 2 +ulysses quirinius 1 +ulysses robinson 1 +ulysses steinbeck 1 +ulysses steinbeck 1 +ulysses thompson 1 +ulysses underhill 1 +ulysses underhill 1 +ulysses underhill 1 +ulysses underhill 1 +ulysses underhill 1 +ulysses underhill 1 +ulysses underhill 1 +ulysses van buren 1 +ulysses white 1 +ulysses white 1 +ulysses xylophone 1 +ulysses xylophone 1 +ulysses xylophone 1 +ulysses young 1 +ulysses young 1 +ulysses young 1 +victor allen 1 +victor allen 1 +victor brown 1 +victor brown 1 +victor brown 1 +victor brown 1 +victor davidson 1 +victor davidson 1 +victor davidson 2 +victor ellison 1 +victor ellison 1 +victor hernandez 1 +victor hernandez 1 +victor hernandez 1 +victor hernandez 1 +victor hernandez 1 +victor johnson 1 +victor johnson 1 +victor johnson 1 +victor king 1 +victor king 1 +victor laertes 1 +victor laertes 1 +victor miller 2 +victor nixon 1 +victor nixon 1 +victor ovid 1 +victor polk 1 +victor quirinius 1 +victor quirinius 1 +victor robinson 1 +victor robinson 1 +victor steinbeck 1 +victor steinbeck 1 +victor steinbeck 1 +victor thompson 1 +victor van buren 1 +victor van buren 1 +victor white 1 +victor white 1 +victor xylophone 1 +victor xylophone 1 +victor xylophone 1 +victor xylophone 1 +victor xylophone 2 +victor young 1 +victor zipper 1 +wendy allen 1 +wendy allen 1 +wendy allen 1 +wendy brown 1 +wendy brown 1 +wendy ellison 1 +wendy ellison 1 +wendy falkner 1 +wendy falkner 1 +wendy falkner 1 +wendy garcia 1 +wendy garcia 1 +wendy garcia 1 +wendy garcia 1 +wendy hernandez 1 +wendy ichabod 1 +wendy king 1 +wendy king 1 +wendy king 1 +wendy laertes 1 +wendy laertes 1 +wendy laertes 1 +wendy miller 1 +wendy miller 1 +wendy nixon 1 +wendy nixon 1 +wendy ovid 1 +wendy ovid 1 +wendy polk 1 +wendy polk 1 +wendy quirinius 1 +wendy quirinius 1 +wendy robinson 1 +wendy robinson 1 +wendy robinson 1 +wendy steinbeck 1 +wendy thompson 1 +wendy thompson 1 +wendy underhill 1 +wendy underhill 1 +wendy underhill 1 +wendy van buren 1 +wendy van buren 1 +wendy white 1 +wendy xylophone 1 +wendy xylophone 1 +wendy young 1 +wendy young 3 +xavier allen 1 +xavier allen 1 +xavier allen 1 +xavier brown 1 +xavier brown 1 +xavier brown 1 +xavier carson 1 +xavier carson 1 +xavier davidson 1 +xavier davidson 1 +xavier davidson 1 +xavier ellison 1 +xavier ellison 1 +xavier garcia 1 +xavier hernandez 1 +xavier hernandez 1 +xavier hernandez 1 +xavier ichabod 1 +xavier ichabod 1 +xavier johnson 1 +xavier johnson 1 +xavier king 1 +xavier king 1 +xavier laertes 1 +xavier ovid 1 +xavier polk 1 +xavier polk 1 +xavier polk 1 +xavier polk 1 +xavier quirinius 1 +xavier quirinius 1 +xavier quirinius 1 +xavier quirinius 1 +xavier thompson 1 +xavier underhill 1 +xavier white 1 +xavier white 1 +xavier xylophone 1 +xavier zipper 2 +yuri allen 1 +yuri allen 1 +yuri brown 1 +yuri brown 1 +yuri carson 1 +yuri carson 1 +yuri ellison 1 +yuri ellison 1 +yuri falkner 1 +yuri falkner 1 +yuri garcia 1 +yuri hernandez 1 +yuri johnson 1 +yuri johnson 1 +yuri johnson 1 +yuri king 1 +yuri laertes 1 +yuri laertes 1 +yuri nixon 1 +yuri nixon 1 +yuri polk 1 +yuri polk 1 +yuri polk 1 +yuri quirinius 1 +yuri quirinius 1 +yuri quirinius 1 +yuri steinbeck 1 +yuri steinbeck 1 +yuri thompson 1 +yuri underhill 1 +yuri underhill 1 +yuri white 1 +yuri xylophone 1 +zach allen 2 +zach brown 1 +zach brown 1 +zach brown 1 +zach brown 1 +zach brown 1 +zach carson 1 +zach ellison 1 +zach falkner 1 +zach falkner 1 +zach garcia 1 +zach garcia 1 +zach garcia 1 +zach garcia 1 +zach ichabod 1 +zach ichabod 1 +zach king 1 +zach king 1 +zach king 2 +zach miller 1 +zach miller 1 +zach miller 1 +zach ovid 1 +zach ovid 1 +zach ovid 1 +zach ovid 1 +zach quirinius 1 +zach robinson 2 +zach steinbeck 2 +zach steinbeck 2 +zach thompson 1 +zach thompson 1 +zach underhill 1 +zach white 1 +zach xylophone 1 +zach xylophone 1 +zach young 1 +zach zipper 1 +zach zipper 1 +zach zipper 1 diff --git a/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 1-1-5c5f373e325115d710a7a23fcb1626f1 b/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 1-1-5c5f373e325115d710a7a23fcb1626f1 new file mode 100644 index 0000000000000..22a6f27253dcf --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 1-1-5c5f373e325115d710a7a23fcb1626f1 @@ -0,0 +1,1049 @@ +zach zipper 4 +zach zipper 3 +zach zipper 1 +zach young 4 +zach xylophone 4 +zach xylophone 1 +zach white 1 +zach underhill 1 +zach thompson 2 +zach thompson 2 +zach steinbeck 5 +zach steinbeck 1 +zach robinson 1 +zach quirinius 3 +zach ovid 5 +zach ovid 4 +zach ovid 3 +zach ovid 1 +zach miller 5 +zach miller 3 +zach miller 1 +zach king 6 +zach king 4 +zach king 1 +zach ichabod 3 +zach ichabod 2 +zach garcia 6 +zach garcia 3 +zach garcia 1 +zach garcia 1 +zach falkner 2 +zach falkner 1 +zach ellison 4 +zach carson 3 +zach brown 5 +zach brown 4 +zach brown 3 +zach brown 2 +zach brown 1 +zach allen 4 +yuri xylophone 3 +yuri white 2 +yuri underhill 6 +yuri underhill 4 +yuri thompson 4 +yuri steinbeck 6 +yuri steinbeck 2 +yuri quirinius 4 +yuri quirinius 3 +yuri quirinius 1 +yuri polk 4 +yuri polk 3 +yuri polk 2 +yuri nixon 3 +yuri nixon 2 +yuri laertes 3 +yuri laertes 1 +yuri king 5 +yuri johnson 4 +yuri johnson 3 +yuri johnson 1 +yuri hernandez 4 +yuri garcia 3 +yuri falkner 7 +yuri falkner 3 +yuri ellison 1 +yuri ellison 1 +yuri carson 7 +yuri carson 4 +yuri brown 3 +yuri brown 1 +yuri allen 3 +yuri allen 2 +xavier zipper 1 +xavier xylophone 1 +xavier white 3 +xavier white 3 +xavier underhill 2 +xavier thompson 3 +xavier quirinius 6 +xavier quirinius 5 +xavier quirinius 2 +xavier quirinius 1 +xavier polk 5 +xavier polk 3 +xavier polk 3 +xavier polk 3 +xavier ovid 5 +xavier laertes 4 +xavier king 3 +xavier king 1 +xavier johnson 3 +xavier johnson 1 +xavier ichabod 2 +xavier ichabod 2 +xavier hernandez 3 +xavier hernandez 1 +xavier hernandez 1 +xavier garcia 4 +xavier ellison 1 +xavier ellison 1 +xavier davidson 5 +xavier davidson 4 +xavier davidson 1 +xavier carson 5 +xavier carson 3 +xavier brown 4 +xavier brown 2 +xavier brown 2 +xavier allen 6 +xavier allen 3 +xavier allen 1 +wendy young 8 +wendy young 2 +wendy xylophone 6 +wendy xylophone 4 +wendy white 5 +wendy van buren 2 +wendy van buren 2 +wendy underhill 6 +wendy underhill 5 +wendy underhill 4 +wendy thompson 5 +wendy thompson 2 +wendy steinbeck 1 +wendy robinson 5 +wendy robinson 3 +wendy robinson 2 +wendy quirinius 6 +wendy quirinius 4 +wendy polk 2 +wendy polk 2 +wendy ovid 4 +wendy ovid 1 +wendy nixon 3 +wendy nixon 1 +wendy miller 2 +wendy miller 1 +wendy laertes 3 +wendy laertes 3 +wendy laertes 1 +wendy king 5 +wendy king 4 +wendy king 1 +wendy ichabod 3 +wendy hernandez 1 +wendy garcia 7 +wendy garcia 5 +wendy garcia 4 +wendy garcia 1 +wendy falkner 3 +wendy falkner 1 +wendy falkner 1 +wendy ellison 2 +wendy ellison 1 +wendy brown 5 +wendy brown 2 +wendy allen 6 +wendy allen 2 +wendy allen 2 +victor zipper 3 +victor young 1 +victor xylophone 6 +victor xylophone 6 +victor xylophone 2 +victor xylophone 1 +victor xylophone 1 +victor white 2 +victor white 1 +victor van buren 4 +victor van buren 4 +victor thompson 2 +victor steinbeck 5 +victor steinbeck 2 +victor steinbeck 1 +victor robinson 4 +victor robinson 2 +victor quirinius 3 +victor quirinius 1 +victor polk 3 +victor ovid 2 +victor nixon 6 +victor nixon 4 +victor miller 1 +victor laertes 4 +victor laertes 3 +victor king 6 +victor king 1 +victor johnson 2 +victor johnson 2 +victor johnson 1 +victor hernandez 6 +victor hernandez 4 +victor hernandez 3 +victor hernandez 1 +victor hernandez 1 +victor ellison 7 +victor ellison 4 +victor davidson 6 +victor davidson 2 +victor davidson 2 +victor brown 4 +victor brown 3 +victor brown 2 +victor brown 1 +victor allen 4 +victor allen 2 +ulysses young 7 +ulysses young 6 +ulysses young 3 +ulysses xylophone 6 +ulysses xylophone 3 +ulysses xylophone 2 +ulysses white 6 +ulysses white 2 +ulysses van buren 3 +ulysses underhill 8 +ulysses underhill 4 +ulysses underhill 3 +ulysses underhill 2 +ulysses underhill 2 +ulysses underhill 1 +ulysses underhill 1 +ulysses thompson 5 +ulysses steinbeck 3 +ulysses steinbeck 1 +ulysses robinson 5 +ulysses quirinius 8 +ulysses polk 6 +ulysses polk 4 +ulysses polk 1 +ulysses polk 1 +ulysses ovid 3 +ulysses nixon 1 +ulysses miller 3 +ulysses miller 2 +ulysses laertes 5 +ulysses laertes 4 +ulysses laertes 2 +ulysses king 2 +ulysses johnson 5 +ulysses ichabod 1 +ulysses ichabod 1 +ulysses hernandez 6 +ulysses hernandez 3 +ulysses hernandez 2 +ulysses garcia 2 +ulysses ellison 2 +ulysses davidson 8 +ulysses carson 4 +ulysses carson 3 +ulysses carson 2 +ulysses carson 1 +ulysses brown 3 +tom zipper 5 +tom young 2 +tom young 1 +tom white 1 +tom van buren 5 +tom van buren 2 +tom van buren 1 +tom steinbeck 4 +tom robinson 8 +tom robinson 4 +tom robinson 3 +tom robinson 2 +tom quirinius 5 +tom quirinius 1 +tom polk 3 +tom polk 2 +tom ovid 2 +tom nixon 5 +tom miller 1 +tom miller 1 +tom miller 1 +tom laertes 4 +tom laertes 2 +tom king 1 +tom johnson 8 +tom johnson 1 +tom ichabod 1 +tom hernandez 3 +tom hernandez 2 +tom falkner 3 +tom falkner 2 +tom ellison 5 +tom ellison 3 +tom ellison 1 +tom davidson 7 +tom carson 3 +tom carson 3 +tom carson 1 +tom brown 4 +tom brown 2 +sarah zipper 1 +sarah young 1 +sarah xylophone 2 +sarah white 4 +sarah white 3 +sarah steinbeck 6 +sarah robinson 3 +sarah robinson 2 +sarah ovid 1 +sarah miller 1 +sarah king 3 +sarah king 2 +sarah johnson 7 +sarah johnson 6 +sarah johnson 4 +sarah johnson 2 +sarah ichabod 4 +sarah ichabod 3 +sarah garcia 2 +sarah garcia 2 +sarah garcia 2 +sarah falkner 7 +sarah falkner 1 +sarah ellison 1 +sarah carson 6 +sarah carson 4 +sarah carson 4 +rachel zipper 8 +rachel zipper 5 +rachel young 3 +rachel white 2 +rachel white 2 +rachel underhill 2 +rachel thompson 5 +rachel thompson 4 +rachel thompson 3 +rachel robinson 10 +rachel robinson 3 +rachel robinson 1 +rachel quirinius 5 +rachel polk 4 +rachel ovid 5 +rachel ovid 4 +rachel laertes 1 +rachel laertes 1 +rachel king 3 +rachel king 1 +rachel johnson 1 +rachel falkner 8 +rachel falkner 5 +rachel falkner 5 +rachel falkner 2 +rachel ellison 6 +rachel davidson 6 +rachel carson 7 +rachel carson 2 +rachel brown 5 +rachel brown 4 +rachel brown 3 +rachel brown 3 +rachel brown 1 +rachel allen 5 +rachel allen 1 +quinn zipper 2 +quinn zipper 2 +quinn young 2 +quinn van buren 2 +quinn underhill 7 +quinn underhill 6 +quinn underhill 2 +quinn thompson 5 +quinn thompson 2 +quinn steinbeck 3 +quinn steinbeck 2 +quinn robinson 2 +quinn quirinius 5 +quinn ovid 6 +quinn nixon 3 +quinn laertes 2 +quinn laertes 2 +quinn laertes 1 +quinn king 2 +quinn king 1 +quinn ichabod 1 +quinn garcia 6 +quinn garcia 3 +quinn garcia 2 +quinn garcia 1 +quinn ellison 7 +quinn ellison 5 +quinn davidson 7 +quinn davidson 4 +quinn davidson 3 +quinn davidson 2 +quinn brown 5 +quinn brown 3 +quinn brown 2 +quinn allen 5 +quinn allen 2 +priscilla zipper 5 +priscilla zipper 2 +priscilla young 4 +priscilla young 1 +priscilla xylophone 7 +priscilla xylophone 2 +priscilla xylophone 1 +priscilla white 4 +priscilla van buren 3 +priscilla van buren 3 +priscilla van buren 2 +priscilla underhill 5 +priscilla underhill 4 +priscilla thompson 2 +priscilla quirinius 4 +priscilla polk 5 +priscilla ovid 4 +priscilla ovid 1 +priscilla nixon 2 +priscilla nixon 1 +priscilla king 4 +priscilla johnson 4 +priscilla johnson 2 +priscilla johnson 2 +priscilla johnson 2 +priscilla johnson 1 +priscilla ichabod 3 +priscilla ichabod 2 +priscilla carson 6 +priscilla carson 5 +priscilla carson 4 +priscilla brown 5 +priscilla brown 5 +priscilla brown 3 +oscar zipper 4 +oscar zipper 4 +oscar zipper 2 +oscar xylophone 7 +oscar xylophone 5 +oscar xylophone 3 +oscar white 5 +oscar white 5 +oscar white 3 +oscar white 2 +oscar van buren 5 +oscar van buren 3 +oscar van buren 2 +oscar underhill 1 +oscar thompson 6 +oscar thompson 3 +oscar thompson 3 +oscar thompson 2 +oscar steinbeck 7 +oscar robinson 7 +oscar robinson 3 +oscar robinson 3 +oscar robinson 1 +oscar quirinius 3 +oscar quirinius 3 +oscar quirinius 2 +oscar quirinius 1 +oscar polk 2 +oscar polk 2 +oscar ovid 4 +oscar ovid 2 +oscar ovid 1 +oscar nixon 1 +oscar laertes 6 +oscar laertes 4 +oscar laertes 3 +oscar laertes 2 +oscar king 4 +oscar king 2 +oscar king 1 +oscar johnson 6 +oscar johnson 3 +oscar ichabod 3 +oscar ichabod 3 +oscar ichabod 1 +oscar ichabod 1 +oscar hernandez 6 +oscar hernandez 6 +oscar garcia 4 +oscar falkner 2 +oscar ellison 2 +oscar ellison 1 +oscar davidson 1 +oscar carson 4 +oscar carson 2 +oscar carson 2 +oscar carson 1 +oscar carson 1 +oscar brown 4 +oscar allen 2 +nick zipper 7 +nick zipper 5 +nick young 4 +nick young 2 +nick xylophone 2 +nick van buren 2 +nick underhill 2 +nick thompson 2 +nick steinbeck 4 +nick robinson 3 +nick robinson 1 +nick quirinius 5 +nick quirinius 1 +nick polk 5 +nick ovid 6 +nick nixon 4 +nick miller 2 +nick laertes 3 +nick johnson 4 +nick johnson 4 +nick ichabod 3 +nick ichabod 3 +nick ichabod 1 +nick garcia 5 +nick garcia 4 +nick garcia 4 +nick falkner 3 +nick falkner 1 +nick ellison 3 +nick ellison 2 +nick davidson 4 +nick brown 3 +nick allen 5 +nick allen 4 +mike zipper 4 +mike zipper 4 +mike zipper 1 +mike young 3 +mike young 1 +mike young 1 +mike white 9 +mike white 7 +mike white 5 +mike white 2 +mike van buren 2 +mike van buren 1 +mike steinbeck 4 +mike steinbeck 2 +mike steinbeck 2 +mike steinbeck 1 +mike quirinius 7 +mike polk 4 +mike polk 2 +mike polk 2 +mike nixon 3 +mike nixon 2 +mike miller 1 +mike king 6 +mike king 5 +mike king 4 +mike king 3 +mike king 1 +mike king 1 +mike ichabod 3 +mike hernandez 2 +mike hernandez 1 +mike garcia 3 +mike garcia 2 +mike garcia 1 +mike falkner 2 +mike ellison 6 +mike ellison 5 +mike ellison 3 +mike ellison 1 +mike ellison 1 +mike davidson 5 +mike davidson 5 +mike carson 9 +mike carson 4 +mike carson 3 +mike brown 2 +mike allen 3 +luke zipper 2 +luke xylophone 1 +luke white 1 +luke van buren 2 +luke underhill 2 +luke underhill 2 +luke underhill 1 +luke thompson 3 +luke robinson 6 +luke robinson 1 +luke quirinius 3 +luke polk 3 +luke polk 1 +luke ovid 3 +luke ovid 1 +luke miller 4 +luke laertes 4 +luke laertes 2 +luke laertes 2 +luke laertes 2 +luke laertes 1 +luke johnson 4 +luke johnson 2 +luke johnson 1 +luke ichabod 4 +luke ichabod 1 +luke garcia 5 +luke garcia 2 +luke falkner 4 +luke falkner 2 +luke ellison 3 +luke ellison 2 +luke ellison 1 +luke davidson 2 +luke davidson 2 +luke brown 5 +luke allen 5 +luke allen 2 +luke allen 1 +luke allen 1 +luke allen 1 +katie zipper 1 +katie zipper 1 +katie young 11 +katie young 6 +katie young 1 +katie xylophone 1 +katie white 5 +katie white 3 +katie van buren 6 +katie van buren 4 +katie robinson 2 +katie polk 5 +katie polk 2 +katie ovid 3 +katie nixon 1 +katie miller 1 +katie miller 1 +katie king 7 +katie king 5 +katie king 4 +katie ichabod 6 +katie ichabod 2 +katie ichabod 1 +katie hernandez 1 +katie garcia 4 +katie garcia 3 +katie falkner 4 +katie ellison 5 +katie ellison 4 +katie davidson 1 +katie brown 6 +katie allen 1 +jessica zipper 7 +jessica zipper 6 +jessica zipper 1 +jessica young 4 +jessica young 3 +jessica xylophone 3 +jessica white 8 +jessica white 6 +jessica white 3 +jessica white 1 +jessica white 1 +jessica van buren 1 +jessica underhill 5 +jessica underhill 3 +jessica underhill 2 +jessica thompson 3 +jessica thompson 2 +jessica robinson 2 +jessica quirinius 4 +jessica quirinius 4 +jessica quirinius 3 +jessica quirinius 1 +jessica polk 4 +jessica ovid 2 +jessica ovid 1 +jessica nixon 3 +jessica nixon 2 +jessica miller 5 +jessica johnson 4 +jessica johnson 3 +jessica ichabod 5 +jessica garcia 4 +jessica garcia 3 +jessica falkner 2 +jessica ellison 5 +jessica ellison 2 +jessica davidson 5 +jessica davidson 2 +jessica davidson 2 +jessica davidson 1 +jessica carson 4 +jessica carson 2 +jessica carson 1 +jessica brown 3 +irene xylophone 3 +irene van buren 2 +irene van buren 1 +irene underhill 5 +irene underhill 1 +irene thompson 6 +irene steinbeck 1 +irene robinson 1 +irene quirinius 6 +irene quirinius 5 +irene quirinius 5 +irene polk 3 +irene polk 2 +irene polk 2 +irene polk 1 +irene polk 1 +irene ovid 6 +irene ovid 6 +irene ovid 5 +irene nixon 4 +irene nixon 4 +irene nixon 1 +irene miller 6 +irene laertes 5 +irene laertes 3 +irene laertes 3 +irene johnson 2 +irene ichabod 7 +irene ichabod 1 +irene garcia 4 +irene garcia 2 +irene garcia 2 +irene falkner 5 +irene falkner 2 +irene ellison 4 +irene ellison 3 +irene carson 1 +irene brown 4 +irene brown 4 +irene brown 1 +irene allen 2 +holly zipper 3 +holly zipper 3 +holly young 2 +holly young 2 +holly xylophone 1 +holly white 3 +holly white 1 +holly van buren 4 +holly underhill 6 +holly underhill 3 +holly underhill 3 +holly underhill 2 +holly thompson 2 +holly thompson 1 +holly thompson 1 +holly robinson 2 +holly polk 7 +holly polk 4 +holly nixon 5 +holly nixon 1 +holly miller 4 +holly laertes 5 +holly king 4 +holly king 1 +holly johnson 2 +holly johnson 2 +holly johnson 2 +holly ichabod 4 +holly ichabod 4 +holly ichabod 2 +holly hernandez 9 +holly hernandez 3 +holly hernandez 3 +holly hernandez 2 +holly falkner 6 +holly brown 3 +holly brown 2 +holly allen 1 +gabriella zipper 5 +gabriella zipper 1 +gabriella young 3 +gabriella young 1 +gabriella white 3 +gabriella van buren 3 +gabriella van buren 1 +gabriella thompson 5 +gabriella thompson 5 +gabriella thompson 5 +gabriella steinbeck 4 +gabriella steinbeck 1 +gabriella polk 4 +gabriella polk 4 +gabriella ovid 2 +gabriella ovid 1 +gabriella miller 1 +gabriella laertes 4 +gabriella king 3 +gabriella king 3 +gabriella ichabod 3 +gabriella ichabod 3 +gabriella ichabod 3 +gabriella ichabod 2 +gabriella ichabod 1 +gabriella hernandez 9 +gabriella hernandez 5 +gabriella garcia 2 +gabriella falkner 4 +gabriella falkner 3 +gabriella falkner 2 +gabriella ellison 3 +gabriella ellison 1 +gabriella davidson 2 +gabriella carson 1 +gabriella brown 6 +gabriella brown 3 +gabriella allen 5 +gabriella allen 3 +fred zipper 1 +fred young 2 +fred young 1 +fred white 3 +fred van buren 7 +fred van buren 1 +fred van buren 1 +fred van buren 1 +fred underhill 4 +fred steinbeck 4 +fred steinbeck 2 +fred steinbeck 1 +fred robinson 3 +fred quirinius 7 +fred quirinius 4 +fred polk 7 +fred polk 6 +fred polk 4 +fred polk 2 +fred nixon 7 +fred nixon 5 +fred nixon 1 +fred nixon 1 +fred miller 1 +fred laertes 4 +fred king 6 +fred king 3 +fred johnson 4 +fred ichabod 3 +fred ichabod 2 +fred hernandez 1 +fred falkner 4 +fred falkner 3 +fred falkner 3 +fred ellison 5 +fred ellison 2 +fred ellison 1 +fred davidson 2 +fred davidson 2 +fred davidson 1 +ethan zipper 2 +ethan zipper 1 +ethan xylophone 3 +ethan white 5 +ethan white 2 +ethan van buren 1 +ethan underhill 1 +ethan robinson 3 +ethan robinson 1 +ethan quirinius 6 +ethan quirinius 2 +ethan quirinius 1 +ethan polk 3 +ethan polk 1 +ethan polk 1 +ethan polk 1 +ethan ovid 2 +ethan nixon 7 +ethan miller 5 +ethan laertes 4 +ethan laertes 4 +ethan laertes 3 +ethan laertes 2 +ethan laertes 2 +ethan laertes 2 +ethan laertes 1 +ethan king 1 +ethan johnson 1 +ethan hernandez 3 +ethan garcia 8 +ethan falkner 2 +ethan falkner 1 +ethan ellison 6 +ethan ellison 4 +ethan carson 6 +ethan brown 4 +ethan brown 3 +ethan brown 3 +ethan brown 1 +ethan brown 1 +ethan brown 1 +ethan allen 4 +david young 4 +david young 1 +david xylophone 6 +david xylophone 4 +david xylophone 1 +david white 2 +david van buren 3 +david van buren 2 +david underhill 7 +david underhill 4 +david underhill 1 +david thompson 1 +david robinson 3 +david robinson 2 +david quirinius 4 +david quirinius 4 +david quirinius 2 +david ovid 4 +david ovid 3 +david nixon 1 +david laertes 4 +david ichabod 6 +david ichabod 3 +david hernandez 7 +david ellison 5 +david ellison 3 +david ellison 3 +david davidson 4 +david davidson 3 +david davidson 1 +david davidson 1 +david brown 6 +david brown 2 +david allen 5 +david allen 2 +calvin zipper 9 +calvin zipper 3 +calvin young 3 +calvin young 1 +calvin xylophone 6 +calvin xylophone 3 +calvin xylophone 1 +calvin white 1 +calvin white 1 +calvin van buren 9 +calvin van buren 1 +calvin underhill 4 +calvin thompson 3 +calvin thompson 2 +calvin steinbeck 6 +calvin steinbeck 3 +calvin steinbeck 3 +calvin robinson 2 +calvin quirinius 4 +calvin quirinius 3 +calvin polk 2 +calvin ovid 5 +calvin ovid 4 +calvin ovid 3 +calvin ovid 1 +calvin nixon 7 +calvin nixon 3 +calvin nixon 2 +calvin laertes 3 +calvin laertes 1 +calvin johnson 2 +calvin hernandez 1 +calvin garcia 3 +calvin falkner 8 +calvin falkner 4 +calvin falkner 4 +calvin falkner 3 +calvin falkner 2 +calvin falkner 1 +calvin ellison 3 +calvin davidson 1 +calvin davidson 1 +calvin carson 1 +calvin brown 5 +calvin brown 3 +calvin brown 1 +calvin allen 1 +bob zipper 4 +bob zipper 1 +bob zipper 1 +bob young 1 +bob xylophone 3 +bob xylophone 2 +bob white 3 +bob white 1 +bob van buren 3 +bob steinbeck 2 +bob quirinius 4 +bob polk 2 +bob ovid 7 +bob ovid 2 +bob ovid 2 +bob ovid 1 +bob miller 1 +bob laertes 5 +bob laertes 1 +bob king 3 +bob king 3 +bob king 2 +bob ichabod 1 +bob hernandez 1 +bob garcia 4 +bob garcia 3 +bob garcia 2 +bob garcia 1 +bob garcia 1 +bob falkner 6 +bob ellison 3 +bob ellison 2 +bob ellison 1 +bob ellison 1 +bob davidson 5 +bob davidson 2 +bob davidson 2 +bob carson 3 +bob brown 8 +bob brown 6 +bob brown 2 +alice zipper 2 +alice zipper 1 +alice zipper 1 +alice xylophone 2 +alice xylophone 2 +alice xylophone 1 +alice van buren 2 +alice underhill 2 +alice steinbeck 7 +alice steinbeck 3 +alice steinbeck 1 +alice robinson 4 +alice robinson 1 +alice quirinius 6 +alice quirinius 4 +alice polk 1 +alice ovid 2 +alice nixon 2 +alice nixon 2 +alice nixon 1 +alice miller 2 +alice laertes 3 +alice laertes 2 +alice king 8 +alice king 4 +alice king 2 +alice johnson 5 +alice hernandez 8 +alice hernandez 8 +alice garcia 1 +alice falkner 5 +alice davidson 2 +alice carson 1 +alice brown 5 +alice allen 5 +alice allen 5 +alice allen 4 + 5 + 4 + 3 diff --git a/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 1-2-ac487cc1b94130bf9ce00e07c7075f65 b/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 1-2-ac487cc1b94130bf9ce00e07c7075f65 new file mode 100644 index 0000000000000..c38e7bbabc21e --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 1-2-ac487cc1b94130bf9ce00e07c7075f65 @@ -0,0 +1,1049 @@ + 0.25047801147227533 + 0.47992351816443596 + 0.6197718631178707 +alice allen 0.7300380228136882 +alice allen 0.8954372623574145 +alice allen 0.9216061185468452 +alice brown 0.22053231939163498 +alice carson 0.2889733840304182 +alice davidson 0.8593155893536122 +alice falkner 0.08604206500956023 +alice garcia 0.2870722433460076 +alice hernandez 0.011472275334608031 +alice hernandez 0.07604562737642585 +alice johnson 0.5181644359464627 +alice king 0.3652007648183556 +alice king 0.8536121673003803 +alice king 0.9771863117870723 +alice laertes 0.870722433460076 +alice laertes 0.870722433460076 +alice miller 0.12045889101338432 +alice nixon 0.4372623574144487 +alice nixon 0.47036328871892924 +alice nixon 0.768642447418738 +alice ovid 0.4665391969407266 +alice polk 0.279467680608365 +alice quirinius 0.8432122370936902 +alice quirinius 0.9923518164435946 +alice robinson 0.5722433460076045 +alice robinson 0.7984790874524715 +alice steinbeck 0.27151051625239003 +alice steinbeck 0.739961759082218 +alice steinbeck 0.9923954372623575 +alice underhill 0.5513307984790875 +alice van buren 0.4923954372623574 +alice xylophone 0.2376425855513308 +alice xylophone 0.26806083650190116 +alice xylophone 0.8776290630975143 +alice zipper 0.33460076045627374 +alice zipper 0.8814531548757171 +alice zipper 0.9445506692160612 +bob brown 0.5038022813688213 +bob brown 0.5066921606118547 +bob brown 0.5372848948374761 +bob carson 0.43346007604562736 +bob davidson 0.21673003802281368 +bob davidson 0.5285171102661597 +bob davidson 0.8413001912045889 +bob ellison 0.2045889101338432 +bob ellison 0.26577437858508607 +bob ellison 0.5793499043977055 +bob ellison 0.9144486692015209 +bob falkner 0.6940726577437859 +bob garcia 0.08555133079847908 +bob garcia 0.17680608365019013 +bob garcia 0.2887189292543021 +bob garcia 0.5418250950570342 +bob garcia 0.5736137667304015 +bob hernandez 0.7813688212927756 +bob ichabod 0.5200764818355641 +bob king 0.0076481835564053535 +bob king 0.5627376425855514 +bob king 0.9524714828897338 +bob laertes 0.32887189292543023 +bob laertes 0.6825095057034221 +bob miller 0.19771863117870722 +bob ovid 0.40304182509505704 +bob ovid 0.40344168260038243 +bob ovid 0.42065009560229444 +bob ovid 0.8403041825095057 +bob polk 0.15019011406844107 +bob quirinius 0.1844106463878327 +bob steinbeck 0.16920152091254753 +bob van buren 0.5086042065009561 +bob white 0.26045627376425856 +bob white 0.7623574144486692 +bob xylophone 0.4474187380497132 +bob xylophone 0.6539923954372624 +bob young 0.4722753346080306 +bob zipper 0.009505703422053232 +bob zipper 0.24091778202676864 +bob zipper 0.4600760456273764 +calvin allen 0.30975143403441685 +calvin brown 0.4448669201520912 +calvin brown 0.5361216730038023 +calvin brown 0.9196940726577438 +calvin carson 0.9315589353612167 +calvin davidson 0.5869980879541109 +calvin davidson 0.6653992395437263 +calvin ellison 0.6977186311787072 +calvin falkner 0.02091254752851711 +calvin falkner 0.03824091778202677 +calvin falkner 0.21223709369024857 +calvin falkner 0.46577946768060835 +calvin falkner 0.5114068441064639 +calvin falkner 0.5950570342205324 +calvin garcia 0.7896749521988528 +calvin hernandez 0.16730038022813687 +calvin johnson 0.9790874524714829 +calvin laertes 0.5487571701720841 +calvin laertes 0.8145315487571702 +calvin nixon 0.019120458891013385 +calvin nixon 0.4467680608365019 +calvin nixon 0.7395437262357415 +calvin ovid 0.14531548757170173 +calvin ovid 0.17490494296577946 +calvin ovid 0.19961977186311788 +calvin ovid 0.9407265774378585 +calvin polk 0.4619771863117871 +calvin quirinius 0.8802281368821293 +calvin quirinius 0.9254302103250478 +calvin robinson 0.13193116634799235 +calvin steinbeck 0.4818355640535373 +calvin steinbeck 0.7418738049713193 +calvin steinbeck 0.8060836501901141 +calvin thompson 0.2179732313575526 +calvin thompson 0.8422053231939164 +calvin underhill 0.7495219885277247 +calvin van buren 0.022813688212927757 +calvin van buren 0.8508604206500956 +calvin white 0.04182509505703422 +calvin white 0.9674952198852772 +calvin xylophone 0.011406844106463879 +calvin xylophone 0.3193116634799235 +calvin xylophone 0.6634799235181644 +calvin young 0.1988527724665392 +calvin young 0.4391634980988593 +calvin zipper 0.5532319391634981 +calvin zipper 0.8726235741444867 +david allen 0.30019120458891013 +david allen 0.3326959847036329 +david brown 0.1338432122370937 +david brown 0.9694072657743786 +david davidson 0.21414913957934992 +david davidson 0.655893536121673 +david davidson 0.7319391634980988 +david davidson 0.8878326996197718 +david ellison 0.6863117870722434 +david ellison 0.6883365200764818 +david ellison 0.7243346007604563 +david hernandez 0.12237093690248566 +david ichabod 0.35564053537284895 +david ichabod 0.7338403041825095 +david laertes 0.3575525812619503 +david nixon 0.33460803059273425 +david ovid 0.3916349809885932 +david ovid 0.6022944550669216 +david quirinius 0.3155893536121673 +david quirinius 0.6577437858508605 +david quirinius 0.9163498098859315 +david robinson 0.6673003802281369 +david robinson 0.6998087954110899 +david thompson 0.25285171102661597 +david underhill 0.1586998087954111 +david underhill 0.35181644359464626 +david underhill 0.7189292543021033 +david van buren 0.05927342256214149 +david van buren 0.5889101338432122 +david white 0.49429657794676807 +david xylophone 0.4875717017208413 +david xylophone 0.6901140684410646 +david xylophone 0.7571701720841301 +david young 0.0019011406844106464 +david young 0.040152963671128104 +ethan allen 0.20532319391634982 +ethan brown 0.10707456978967496 +ethan brown 0.13307984790874525 +ethan brown 0.4340344168260038 +ethan brown 0.4752851711026616 +ethan brown 0.5219885277246654 +ethan brown 0.745697896749522 +ethan carson 0.20912547528517111 +ethan ellison 0.45124282982791586 +ethan ellison 0.8680688336520076 +ethan falkner 0.0994263862332696 +ethan falkner 0.6845124282982792 +ethan garcia 0.06653992395437262 +ethan hernandez 0.2237093690248566 +ethan johnson 0.2300380228136882 +ethan king 0.47418738049713194 +ethan laertes 0.022944550669216062 +ethan laertes 0.2908745247148289 +ethan laertes 0.42638623326959846 +ethan laertes 0.48098859315589354 +ethan laertes 0.6596558317399618 +ethan laertes 0.7839388145315488 +ethan laertes 0.9201520912547528 +ethan miller 0.23709369024856597 +ethan nixon 0.8164435946462715 +ethan ovid 0.6121673003802282 +ethan polk 0.12167300380228137 +ethan polk 0.3384321223709369 +ethan polk 0.6920152091254753 +ethan polk 0.9619771863117871 +ethan quirinius 0.19391634980988592 +ethan quirinius 0.23135755258126195 +ethan quirinius 0.7908745247148289 +ethan robinson 0.24282982791587 +ethan robinson 0.8003802281368821 +ethan underhill 0.6615969581749049 +ethan van buren 0.8365019011406845 +ethan white 0.48859315589353614 +ethan white 0.5741444866920152 +ethan xylophone 0.9695817490494296 +ethan zipper 0.21102661596958175 +ethan zipper 0.6425855513307985 +fred davidson 0.5239005736137667 +fred davidson 0.7414448669201521 +fred davidson 0.8604206500956023 +fred ellison 0.3977055449330784 +fred ellison 0.5506692160611855 +fred ellison 0.7208413001912046 +fred falkner 0.024714828897338403 +fred falkner 0.19120458891013384 +fred falkner 0.9809885931558935 +fred hernandez 0.2734225621414914 +fred ichabod 0.17110266159695817 +fred ichabod 0.780114722753346 +fred johnson 0.30038022813688214 +fred king 0.2198852772466539 +fred king 0.47718631178707227 +fred laertes 0.2332695984703633 +fred miller 0.7858508604206501 +fred nixon 0.005703422053231939 +fred nixon 0.31749049429657794 +fred nixon 0.7648183556405354 +fred nixon 0.8460076045627376 +fred polk 0.16252390057361377 +fred polk 0.564638783269962 +fred polk 0.6273764258555133 +fred polk 0.8155893536121673 +fred quirinius 0.4866920152091255 +fred quirinius 0.8973384030418251 +fred robinson 0.6387832699619772 +fred steinbeck 0.14722753346080306 +fred steinbeck 0.4627151051625239 +fred steinbeck 0.7265774378585086 +fred underhill 0.35361216730038025 +fred van buren 0.3365200764818356 +fred van buren 0.5057034220532319 +fred van buren 0.6463878326996197 +fred van buren 0.904397705544933 +fred white 0.5171102661596958 +fred young 0.7705544933078394 +fred young 0.7992351816443595 +fred zipper 0.615678776290631 +gabriella allen 0.4435946462715105 +gabriella allen 0.9334600760456274 +gabriella brown 0.4359464627151052 +gabriella brown 0.9636711281070746 +gabriella carson 0.9562737642585551 +gabriella davidson 0.8174904942965779 +gabriella ellison 0.1931166347992352 +gabriella ellison 0.38022813688212925 +gabriella falkner 0.3231939163498099 +gabriella falkner 0.5659655831739961 +gabriella falkner 0.8948374760994264 +gabriella garcia 0.4695817490494297 +gabriella hernandez 0.6444866920152091 +gabriella hernandez 0.7015209125475285 +gabriella ichabod 0.09125475285171103 +gabriella ichabod 0.1520912547528517 +gabriella ichabod 0.1835564053537285 +gabriella ichabod 0.372848948374761 +gabriella ichabod 0.8107074569789675 +gabriella king 0.39961759082217974 +gabriella king 0.5190114068441065 +gabriella laertes 0.4569789674952199 +gabriella miller 0.26996197718631176 +gabriella ovid 0.7091254752851711 +gabriella ovid 0.8897338403041825 +gabriella polk 0.030418250950570342 +gabriella polk 0.44106463878326996 +gabriella steinbeck 0.5755258126195029 +gabriella steinbeck 0.8221797323135756 +gabriella thompson 0.013307984790874524 +gabriella thompson 0.44866920152091255 +gabriella thompson 0.7224334600760456 +gabriella van buren 0.6216730038022814 +gabriella van buren 0.6730038022813688 +gabriella white 0.17208413001912046 +gabriella young 0.5076045627376425 +gabriella young 0.7934990439770554 +gabriella zipper 0.23193916349809887 +gabriella zipper 0.8565965583173997 +holly allen 0.11596958174904944 +holly brown 0.11281070745697896 +holly brown 0.155893536121673 +holly falkner 0.124282982791587 +holly hernandez 0.055449330783938815 +holly hernandez 0.32509505703422054 +holly hernandez 0.97131931166348 +holly hernandez 0.9714828897338403 +holly ichabod 0.12357414448669202 +holly ichabod 0.17300380228136883 +holly ichabod 0.629277566539924 +holly johnson 0.33078393881453155 +holly johnson 0.8612167300380228 +holly johnson 0.9391634980988594 +holly king 0.25475285171102663 +holly king 0.3745247148288973 +holly laertes 0.42775665399239543 +holly miller 0.37476099426386233 +holly nixon 0.10076045627376426 +holly nixon 0.34608030592734224 +holly polk 0.40535372848948376 +holly polk 0.5209125475285171 +holly robinson 0.9273422562141491 +holly thompson 0.1596958174904943 +holly thompson 0.311787072243346 +holly thompson 0.9125475285171103 +holly underhill 0.3479923518164436 +holly underhill 0.5812619502868069 +holly underhill 0.8384030418250951 +holly underhill 0.903041825095057 +holly van buren 0.9464627151051626 +holly white 0.1089866156787763 +holly white 0.4780114722753346 +holly xylophone 0.5304182509505704 +holly young 0.7357414448669202 +holly young 0.8240917782026769 +holly zipper 0.15399239543726237 +holly zipper 0.8546845124282982 +irene allen 0.8738049713193117 +irene brown 0.4588910133843212 +irene brown 0.49619771863117873 +irene brown 0.5678776290630975 +irene carson 0.6844106463878327 +irene ellison 0.32504780114722753 +irene ellison 0.48565965583174 +irene falkner 0.41825095057034223 +irene falkner 0.9866920152091255 +irene garcia 0.11663479923518165 +irene garcia 0.29277566539923955 +irene garcia 0.8126195028680688 +irene ichabod 0.8307984790874525 +irene ichabod 0.9177820267686424 +irene johnson 0.7112810707456979 +irene laertes 0.01338432122370937 +irene laertes 0.1482889733840304 +irene laertes 0.7034220532319392 +irene miller 0.367112810707457 +irene nixon 0.11854684512428298 +irene nixon 0.7927756653992395 +irene nixon 0.9426386233269598 +irene ovid 0.24714828897338403 +irene ovid 0.30210325047801145 +irene ovid 0.779467680608365 +irene polk 0.0038022813688212928 +irene polk 0.45315487571701724 +irene polk 0.6577946768060836 +irene polk 0.8891013384321224 +irene polk 0.9789674952198852 +irene quirinius 0.27533460803059273 +irene quirinius 0.35946462715105165 +irene quirinius 0.384321223709369 +irene robinson 0.18631178707224336 +irene steinbeck 0.9942965779467681 +irene thompson 0.6939163498098859 +irene underhill 0.30401529636711283 +irene underhill 0.3403041825095057 +irene van buren 0.5908221797323135 +irene van buren 0.6634980988593155 +irene xylophone 0.5342205323193916 +jessica brown 0.7680608365019012 +jessica carson 0.3574144486692015 +jessica carson 0.6195028680688337 +jessica carson 0.8269961977186312 +jessica davidson 0.10646387832699619 +jessica davidson 0.34790874524714827 +jessica davidson 0.3593155893536122 +jessica davidson 0.6768060836501901 +jessica ellison 0.0779467680608365 +jessica ellison 0.42015209125475284 +jessica falkner 0.994263862332696 +jessica garcia 0.8279158699808795 +jessica garcia 0.9581749049429658 +jessica ichabod 0.45627376425855515 +jessica johnson 0.30228136882129275 +jessica johnson 0.8049713193116634 +jessica miller 0.8011472275334608 +jessica nixon 0.06500956022944551 +jessica nixon 0.6042065009560229 +jessica ovid 0.15105162523900573 +jessica ovid 0.8992395437262357 +jessica polk 0.4378585086042065 +jessica quirinius 0.058935361216730035 +jessica quirinius 0.4714828897338403 +jessica quirinius 0.5760456273764258 +jessica quirinius 0.8935361216730038 +jessica robinson 0.9638783269961977 +jessica thompson 0.08221797323135756 +jessica thompson 0.5893536121673004 +jessica underhill 0.034220532319391636 +jessica underhill 0.06118546845124283 +jessica underhill 0.9541108986615678 +jessica van buren 0.20650095602294455 +jessica white 0.06273764258555133 +jessica white 0.4149139579349904 +jessica white 0.5798479087452472 +jessica white 0.591254752851711 +jessica white 0.7667304015296367 +jessica xylophone 0.5009560229445507 +jessica young 0.3403441682600382 +jessica young 0.8821292775665399 +jessica zipper 0.14068441064638784 +jessica zipper 0.2984790874524715 +jessica zipper 0.6007604562737643 +katie allen 0.5665399239543726 +katie brown 0.49521988527724664 +katie davidson 0.6730401529636711 +katie ellison 0.3173996175908222 +katie ellison 0.7262357414448669 +katie falkner 0.2676864244741874 +katie garcia 0.049429657794676805 +katie garcia 0.3135755258126195 +katie hernandez 0.6026615969581749 +katie ichabod 0.15296367112810708 +katie ichabod 0.4684512428298279 +katie ichabod 0.7055449330783938 +katie king 0.16159695817490494 +katie king 0.502868068833652 +katie king 0.5927342256214149 +katie miller 0.5228136882129277 +katie miller 0.5296367112810707 +katie nixon 0.7832699619771863 +katie ovid 0.8795411089866156 +katie polk 0.35372848948374763 +katie polk 0.9657794676806084 +katie robinson 0.06844106463878327 +katie van buren 0.06883365200764818 +katie van buren 0.1739961759082218 +katie white 0.045889101338432124 +katie white 0.18546845124282982 +katie xylophone 0.7281368821292775 +katie young 0.16443594646271512 +katie young 0.20152091254752852 +katie young 0.9732313575525813 +katie zipper 0.21863117870722434 +katie zipper 0.4505703422053232 +luke allen 0.03612167300380228 +luke allen 0.21606118546845124 +luke allen 0.8346007604562737 +luke allen 0.8631178707224335 +luke allen 0.9311663479923518 +luke brown 0.7304015296367112 +luke davidson 0.25239005736137665 +luke davidson 0.9961977186311787 +luke ellison 0.1147227533460803 +luke ellison 0.2447418738049713 +luke ellison 0.49809885931558934 +luke falkner 0.24524714828897337 +luke falkner 0.5124282982791587 +luke garcia 0.03441682600382409 +luke garcia 0.32695984703632885 +luke ichabod 0.10266159695817491 +luke ichabod 0.5551330798479087 +luke johnson 0.25430210325047803 +luke johnson 0.6787762906309751 +luke johnson 0.9082217973231358 +luke laertes 0.06309751434034416 +luke laertes 0.3690248565965583 +luke laertes 0.7743785850860421 +luke laertes 0.8079847908745247 +luke laertes 0.811787072243346 +luke miller 0.8068833652007649 +luke ovid 0.435361216730038 +luke ovid 0.7547528517110266 +luke polk 0.13957934990439771 +luke polk 0.9770554493307839 +luke quirinius 0.09315589353612168 +luke robinson 0.015209125475285171 +luke robinson 0.053231939163498096 +luke thompson 0.8840304182509505 +luke underhill 0.08745247148288973 +luke underhill 0.40152963671128106 +luke underhill 0.4608030592734226 +luke van buren 0.4847908745247148 +luke white 0.8098859315589354 +luke xylophone 0.34220532319391633 +luke zipper 0.21292775665399238 +mike allen 0.7036328871892925 +mike brown 0.29063097514340347 +mike carson 0.623574144486692 +mike carson 0.7476099426386233 +mike carson 0.9885931558935361 +mike davidson 0.6520912547528517 +mike davidson 0.8298279158699808 +mike ellison 0.24665391969407266 +mike ellison 0.3821292775665399 +mike ellison 0.8355640535372849 +mike ellison 0.8986615678776291 +mike ellison 0.94106463878327 +mike falkner 0.0248565965583174 +mike garcia 0.39543726235741444 +mike garcia 0.5391969407265774 +mike garcia 0.6482889733840305 +mike hernandez 0.07984790874524715 +mike hernandez 0.7186311787072244 +mike ichabod 0.7642585551330798 +mike king 0.09695817490494296 +mike king 0.188212927756654 +mike king 0.4049429657794677 +mike king 0.5544933078393881 +mike king 0.6045627376425855 +mike king 0.9011406844106464 +mike miller 0.621414913957935 +mike nixon 0.688212927756654 +mike nixon 0.9068441064638784 +mike polk 0.3612167300380228 +mike polk 0.6749521988527725 +mike polk 0.8374760994263862 +mike quirinius 0.5105162523900574 +mike steinbeck 0.05736137667304015 +mike steinbeck 0.747148288973384 +mike steinbeck 0.8745247148288974 +mike steinbeck 0.9330783938814532 +mike van buren 0.8650190114068441 +mike van buren 0.973384030418251 +mike white 0.17782026768642448 +mike white 0.7151051625239006 +mike white 0.7566539923954373 +mike white 0.9808795411089866 +mike young 0.20722433460076045 +mike young 0.3840304182509506 +mike young 0.6405353728489483 +mike zipper 0.12810707456978968 +mike zipper 0.42829827915869984 +mike zipper 0.7946768060836502 +nick allen 0.021032504780114723 +nick allen 0.847036328871893 +nick brown 0.14258555133079848 +nick davidson 0.26003824091778205 +nick ellison 0.028680688336520075 +nick ellison 0.3935361216730038 +nick falkner 0.5684410646387833 +nick falkner 0.7590822179732314 +nick garcia 0.34980988593155893 +nick garcia 0.45817490494296575 +nick garcia 0.892925430210325 +nick ichabod 0.2944550669216061 +nick ichabod 0.37667304015296366 +nick ichabod 0.7074569789674953 +nick johnson 0.3973384030418251 +nick johnson 0.4646271510516252 +nick laertes 0.36311787072243346 +nick miller 0.9961759082217974 +nick nixon 0.7110266159695817 +nick ovid 0.7762906309751434 +nick polk 1.0 +nick quirinius 0.0019120458891013384 +nick quirinius 0.08795411089866156 +nick robinson 0.09505703422053231 +nick robinson 0.45506692160611856 +nick steinbeck 0.2224334600760456 +nick thompson 0.4225621414913958 +nick underhill 0.9101338432122371 +nick van buren 0.03802281368821293 +nick xylophone 0.6806883365200764 +nick young 0.4220532319391635 +nick young 0.8623326959847036 +nick zipper 0.2829827915869981 +nick zipper 0.5468451242829828 +oscar allen 0.785171102661597 +oscar brown 0.13498098859315588 +oscar carson 0.07224334600760456 +oscar carson 0.25665399239543724 +oscar carson 0.3422562141491396 +oscar carson 0.6061185468451242 +oscar carson 0.6826003824091779 +oscar davidson 0.7129277566539924 +oscar ellison 0.036328871892925434 +oscar ellison 0.5831739961759083 +oscar falkner 0.9049429657794676 +oscar garcia 0.02676864244741874 +oscar hernandez 0.20076481835564053 +oscar hernandez 0.7870722433460076 +oscar ichabod 0.12619502868068833 +oscar ichabod 0.14149139579349904 +oscar ichabod 0.4416826003824092 +oscar ichabod 0.8661567877629063 +oscar johnson 0.1806083650190114 +oscar johnson 0.467680608365019 +oscar king 0.6596958174904943 +oscar king 0.6787072243346007 +oscar king 0.9258555133079848 +oscar laertes 0.24904942965779467 +oscar laertes 0.5315487571701721 +oscar laertes 0.6328871892925431 +oscar laertes 0.9980988593155894 +oscar nixon 0.9292543021032504 +oscar ovid 0.43021032504780116 +oscar ovid 0.8288973384030418 +oscar ovid 0.8527724665391969 +oscar polk 0.10836501901140684 +oscar polk 0.37858508604206503 +oscar quirinius 0.3041825095057034 +oscar quirinius 0.46387832699619774 +oscar quirinius 0.6311787072243346 +oscar quirinius 0.8555133079847909 +oscar robinson 0.11216730038022814 +oscar robinson 0.22433460076045628 +oscar robinson 0.2294455066921606 +oscar robinson 0.2390057361376673 +oscar steinbeck 0.9904942965779467 +oscar thompson 0.015296367112810707 +oscar thompson 0.2946768060836502 +oscar thompson 0.3060836501901141 +oscar thompson 0.6140684410646388 +oscar underhill 0.31368821292775667 +oscar van buren 0.722753346080306 +oscar van buren 0.7889733840304183 +oscar van buren 0.8833652007648184 +oscar white 0.055133079847908745 +oscar white 0.22562141491395793 +oscar white 0.4321223709369025 +oscar white 0.6443594646271511 +oscar xylophone 0.10133843212237094 +oscar xylophone 0.4187380497131931 +oscar xylophone 0.4296577946768061 +oscar zipper 0.6233269598470363 +oscar zipper 0.7490494296577946 +oscar zipper 0.8783269961977186 +priscilla brown 0.2925430210325048 +priscilla brown 0.6501901140684411 +priscilla brown 0.9120458891013384 +priscilla carson 0.22753346080305928 +priscilla carson 0.5564053537284895 +priscilla carson 0.7820267686424475 +priscilla ichabod 0.3269961977186312 +priscilla ichabod 0.9828897338403042 +priscilla johnson 0.04206500956022945 +priscilla johnson 0.4011406844106464 +priscilla johnson 0.6368821292775665 +priscilla johnson 0.7131931166347992 +priscilla johnson 0.9429657794676806 +priscilla king 0.3517110266159696 +priscilla nixon 0.38049713193116635 +priscilla nixon 0.6864244741873805 +priscilla ovid 0.8193916349809885 +priscilla ovid 0.9139579349904398 +priscilla polk 0.5697896749521989 +priscilla quirinius 0.22179732313575526 +priscilla thompson 0.7737642585551331 +priscilla underhill 0.1682600382409178 +priscilla underhill 0.8852772466539197 +priscilla van buren 0.10325047801147227 +priscilla van buren 0.7877629063097514 +priscilla van buren 0.9598470363288719 +priscilla white 0.4894837476099426 +priscilla xylophone 0.596958174904943 +priscilla xylophone 0.6159695817490495 +priscilla xylophone 0.8393881453154876 +priscilla young 0.41064638783269963 +priscilla young 0.9182509505703422 +priscilla zipper 0.5247148288973384 +priscilla zipper 0.8574144486692015 +quinn allen 0.1634980988593156 +quinn allen 0.9617590822179732 +quinn brown 0.08986615678776291 +quinn brown 0.17590822179732313 +quinn brown 0.5836501901140685 +quinn davidson 0.11787072243346007 +quinn davidson 0.30592734225621415 +quinn davidson 0.3650190114068441 +quinn davidson 0.751434034416826 +quinn ellison 0.376425855513308 +quinn ellison 0.8517110266159695 +quinn garcia 0.17870722433460076 +quinn garcia 0.7323135755258127 +quinn garcia 0.844106463878327 +quinn garcia 0.9486692015209125 +quinn ichabod 0.42395437262357416 +quinn king 0.6653919694072657 +quinn king 0.9505703422053232 +quinn laertes 0.6080305927342257 +quinn laertes 0.9277566539923955 +quinn laertes 0.9847908745247148 +quinn nixon 0.5133079847908745 +quinn ovid 0.16539923954372623 +quinn quirinius 0.19011406844106463 +quinn robinson 0.27756653992395436 +quinn steinbeck 0.23954372623574144 +quinn steinbeck 0.6367112810707457 +quinn thompson 0.4068441064638783 +quinn thompson 0.7782026768642447 +quinn underhill 0.05353728489483748 +quinn underhill 0.5380228136882129 +quinn underhill 0.9349904397705545 +quinn van buren 0.2623574144486692 +quinn young 0.37832699619771865 +quinn zipper 0.51434034416826 +quinn zipper 0.8859315589353612 +rachel allen 0.1701720841300191 +rachel allen 0.3288973384030418 +rachel brown 0.04780114722753346 +rachel brown 0.057034220532319393 +rachel brown 0.532319391634981 +rachel brown 0.5946462715105163 +rachel brown 0.6064638783269962 +rachel carson 0.09177820267686425 +rachel carson 0.6406844106463878 +rachel davidson 0.37093690248565964 +rachel ellison 0.5162523900573613 +rachel falkner 0.1958174904942966 +rachel falkner 0.6330798479087453 +rachel falkner 0.6768642447418738 +rachel falkner 0.9751434034416826 +rachel johnson 0.9560229445506692 +rachel king 0.12547528517110265 +rachel king 0.6003824091778203 +rachel laertes 0.2638623326959847 +rachel laertes 0.5779467680608364 +rachel ovid 0.23518164435946462 +rachel ovid 0.7053231939163498 +rachel polk 0.14638783269961977 +rachel quirinius 0.0076045627376425855 +rachel robinson 0.14340344168260039 +rachel robinson 0.2084130019120459 +rachel robinson 0.6902485659655831 +rachel thompson 0.2718631178707224 +rachel thompson 0.5334608030592735 +rachel thompson 0.875717017208413 +rachel underhill 0.344106463878327 +rachel white 0.17973231357552583 +rachel white 0.6615678776290631 +rachel young 0.3862332695984704 +rachel zipper 0.33079847908745247 +rachel zipper 0.5717017208413002 +sarah carson 0.08935361216730038 +sarah carson 0.18250950570342206 +sarah carson 0.8041825095057035 +sarah ellison 0.967680608365019 +sarah falkner 0.6252390057361377 +sarah falkner 0.9024856596558317 +sarah garcia 0.3881453154875717 +sarah garcia 0.4072657743785851 +sarah garcia 0.6673040152963671 +sarah ichabod 0.29636711281070743 +sarah ichabod 0.9483747609942639 +sarah johnson 0.06463878326996197 +sarah johnson 0.10456273764258556 +sarah johnson 0.5640535372848948 +sarah johnson 0.7954110898661568 +sarah king 0.8030592734225621 +sarah king 0.9655831739961759 +sarah miller 0.6692160611854685 +sarah ovid 0.20342205323193915 +sarah robinson 0.47338403041825095 +sarah robinson 0.7775665399239544 +sarah steinbeck 0.6520076481835564 +sarah white 0.28517110266159695 +sarah white 0.8479087452471483 +sarah xylophone 0.25621414913957935 +sarah young 0.5570342205323194 +sarah zipper 0.5583173996175909 +tom brown 0.5602294455066922 +tom brown 0.8669201520912547 +tom carson 0.045627376425855515 +tom carson 0.35551330798479086 +tom carson 0.935361216730038 +tom davidson 0.8212927756653993 +tom ellison 0.21032504780114722 +tom ellison 0.26195028680688337 +tom ellison 0.7376425855513308 +tom falkner 0.3441682600382409 +tom falkner 0.6481835564053537 +tom hernandez 0.0038240917782026767 +tom hernandez 0.5399239543726235 +tom ichabod 0.6137667304015296 +tom johnson 0.5525812619502868 +tom johnson 0.7915869980879541 +tom king 0.16061185468451242 +tom laertes 0.0745697896749522 +tom laertes 0.5095057034220533 +tom miller 0.2262357414448669 +tom miller 0.2338403041825095 +tom miller 0.2813688212927757 +tom nixon 0.8451242829827916 +tom ovid 0.864244741873805 +tom polk 0.1491395793499044 +tom polk 0.9521988527724665 +tom quirinius 0.09369024856596558 +tom quirinius 0.8489483747609943 +tom robinson 0.060836501901140684 +tom robinson 0.6254752851711026 +tom robinson 0.6462715105162524 +tom robinson 0.9980879541108987 +tom steinbeck 0.5817490494296578 +tom van buren 0.12737642585551331 +tom van buren 0.3154875717017208 +tom van buren 0.7585551330798479 +tom white 0.47609942638623326 +tom young 0.9369024856596558 +tom young 0.9543726235741445 +tom zipper 0.9063097514340345 +ulysses brown 0.9448669201520913 +ulysses carson 0.07034220532319392 +ulysses carson 0.09885931558935361 +ulysses carson 0.2414448669201521 +ulysses carson 0.7604562737642585 +ulysses davidson 0.7093690248565966 +ulysses ellison 0.55893536121673 +ulysses garcia 0.7246653919694073 +ulysses hernandez 0.4091778202676864 +ulysses hernandez 0.627151051625239 +ulysses hernandez 0.982791586998088 +ulysses ichabod 0.21482889733840305 +ulysses ichabod 0.3193916349809886 +ulysses johnson 0.5621414913957935 +ulysses king 0.9467680608365019 +ulysses laertes 0.390057361376673 +ulysses laertes 0.7973231357552581 +ulysses laertes 0.9866156787762906 +ulysses miller 0.31166347992351817 +ulysses miller 0.5774378585086042 +ulysses nixon 0.0057361376673040155 +ulysses ovid 0.38593155893536124 +ulysses polk 0.04752851711026616 +ulysses polk 0.6083650190114068 +ulysses polk 0.7609942638623327 +ulysses polk 0.8326996197718631 +ulysses quirinius 0.6290630975143403 +ulysses robinson 0.9235181644359465 +ulysses steinbeck 0.039923954372623575 +ulysses steinbeck 0.7724665391969407 +ulysses thompson 0.3824091778202677 +ulysses underhill 0.11406844106463879 +ulysses underhill 0.23574144486692014 +ulysses underhill 0.3365019011406844 +ulysses underhill 0.42585551330798477 +ulysses underhill 0.6102661596958175 +ulysses underhill 0.6959847036328872 +ulysses underhill 0.9752851711026616 +ulysses van buren 0.5437262357414449 +ulysses white 0.5 +ulysses white 0.5931558935361216 +ulysses xylophone 0.5855513307984791 +ulysses xylophone 0.8317399617590823 +ulysses xylophone 0.9005736137667304 +ulysses young 0.18164435946462715 +ulysses young 0.3919694072657744 +ulysses young 0.49049429657794674 +victor allen 0.13575525812619502 +victor allen 0.6309751434034416 +victor brown 0.0497131931166348 +victor brown 0.20267686424474188 +victor brown 0.6178707224334601 +victor brown 0.8910133843212237 +victor davidson 0.026615969581749048 +victor davidson 0.491395793499044 +victor davidson 0.5850860420650096 +victor ellison 0.26425855513307983 +victor ellison 0.6692015209125475 +victor hernandez 0.04397705544933078 +victor hernandez 0.12927756653992395 +victor hernandez 0.1950286806883365 +victor hernandez 0.5411089866156787 +victor hernandez 0.7284894837476099 +victor johnson 0.11977186311787072 +victor johnson 0.4828897338403042 +victor johnson 0.7699619771863118 +victor king 0.41254752851711024 +victor king 0.714828897338403 +victor laertes 0.43155893536121676 +victor laertes 0.6500956022944551 +victor miller 0.4429657794676806 +victor nixon 0.33269961977186313 +victor nixon 0.5258126195028681 +victor ovid 0.22813688212927757 +victor polk 0.13878326996197718 +victor quirinius 0.13766730401529637 +victor quirinius 0.887189292543021 +victor robinson 0.5494296577946768 +victor robinson 0.7509505703422054 +victor steinbeck 0.08365019011406843 +victor steinbeck 0.15487571701720843 +victor steinbeck 0.3669201520912547 +victor thompson 0.10516252390057361 +victor van buren 0.27724665391969405 +victor van buren 0.9579349904397706 +victor white 0.41634980988593157 +victor white 0.6349809885931559 +victor xylophone 0.13688212927756654 +victor xylophone 0.3078393881453155 +victor xylophone 0.4110898661567878 +victor xylophone 0.5449330783938815 +victor xylophone 0.9296577946768061 +victor young 0.18738049713193117 +victor zipper 0.5430210325047801 +wendy allen 0.3231357552581262 +wendy allen 0.734225621414914 +wendy allen 0.869980879541109 +wendy brown 0.18929254302103252 +wendy brown 0.6996197718631179 +wendy ellison 0.7437858508604207 +wendy ellison 0.8498098859315589 +wendy falkner 0.07648183556405354 +wendy falkner 0.5353728489483748 +wendy falkner 0.7756653992395437 +wendy garcia 0.07074569789674952 +wendy garcia 0.0741444866920152 +wendy garcia 0.33840304182509506 +wendy garcia 0.38783269961977185 +wendy hernandez 0.017110266159695818 +wendy ichabod 0.8718929254302104 +wendy king 0.37072243346007605 +wendy king 0.497131931166348 +wendy king 0.5965583173996176 +wendy laertes 0.32122370936902483 +wendy laertes 0.49904397705544934 +wendy laertes 0.876425855513308 +wendy miller 0.7533460803059273 +wendy miller 0.7552581261950286 +wendy nixon 0.44933078393881454 +wendy nixon 0.7661596958174905 +wendy ovid 0.5019011406844106 +wendy ovid 0.6978967495219885 +wendy polk 0.3688212927756654 +wendy polk 0.526615969581749 +wendy quirinius 0.1444866920152091 +wendy quirinius 0.5874524714828897 +wendy robinson 0.030592734225621414 +wendy robinson 0.06692160611854685 +wendy robinson 0.27566539923954375 +wendy steinbeck 0.5703422053231939 +wendy thompson 0.028517110266159697 +wendy thompson 0.11089866156787763 +wendy underhill 0.4837476099426386 +wendy underhill 0.6424474187380497 +wendy underhill 0.9600760456273765 +wendy van buren 0.1920152091254753 +wendy van buren 0.7433460076045627 +wendy white 0.752851711026616 +wendy xylophone 0.6347992351816444 +wendy xylophone 0.7452471482889734 +wendy young 0.07839388145315487 +wendy young 0.3897338403041825 +xavier allen 0.043726235741444866 +xavier allen 0.361376673040153 +xavier allen 0.5456273764258555 +xavier brown 0.6711281070745698 +xavier brown 0.9158699808795411 +xavier brown 0.9847036328871893 +xavier carson 0.0841300191204589 +xavier carson 0.988527724665392 +xavier davidson 0.2585551330798479 +xavier davidson 0.4168260038240918 +xavier davidson 0.609942638623327 +xavier ellison 0.5984703632887189 +xavier ellison 0.7361376673040153 +xavier garcia 0.7017208413001912 +xavier hernandez 0.2509505703422053 +xavier hernandez 0.34990439770554493 +xavier hernandez 0.9220532319391636 +xavier ichabod 0.5475285171102662 +xavier ichabod 0.858508604206501 +xavier johnson 0.3938814531548757 +xavier johnson 0.8231939163498099 +xavier king 0.03231939163498099 +xavier king 0.6539196940726577 +xavier laertes 0.5988593155893536 +xavier ovid 0.4397705544933078 +xavier polk 0.4933078393881453 +xavier polk 0.762906309751434 +xavier polk 0.8136882129277566 +xavier polk 0.8260038240917782 +xavier quirinius 0.07265774378585087 +xavier quirinius 0.27915869980879543 +xavier quirinius 0.34600760456273766 +xavier quirinius 0.8022813688212928 +xavier thompson 0.6118546845124283 +xavier underhill 0.16634799235181644 +xavier white 0.6958174904942965 +xavier white 0.7380497131931166 +xavier xylophone 0.8183556405353728 +xavier zipper 0.9904397705544933 +yuri allen 0.9106463878326996 +yuri allen 1.0 +yuri brown 0.5152091254752852 +yuri brown 0.908745247148289 +yuri carson 0.09560229445506692 +yuri carson 0.9372623574144486 +yuri ellison 0.017208413001912046 +yuri ellison 0.39923954372623577 +yuri falkner 0.28680688336520077 +yuri falkner 0.8967495219885278 +yuri garcia 0.2661596958174905 +yuri hernandez 0.28489483747609945 +yuri johnson 0.5047801147227533 +yuri johnson 0.655831739961759 +yuri johnson 0.720532319391635 +yuri king 0.32129277566539927 +yuri laertes 0.4144486692015209 +yuri laertes 0.8916349809885932 +yuri nixon 0.05162523900573614 +yuri nixon 0.40874524714828897 +yuri polk 0.051330798479087454 +yuri polk 0.39579349904397704 +yuri polk 0.6749049429657795 +yuri quirinius 0.08030592734225621 +yuri quirinius 0.2982791586998088 +yuri quirinius 0.4130019120458891 +yuri steinbeck 0.15779467680608364 +yuri steinbeck 0.9388145315487572 +yuri thompson 0.6175908221797323 +yuri underhill 0.42447418738049714 +yuri underhill 0.8202676864244742 +yuri white 0.19694072657743786 +yuri xylophone 0.4790874524714829 +zach allen 0.8250950570342205 +zach brown 0.0817490494296578 +zach brown 0.09751434034416825 +zach brown 0.248565965583174 +zach brown 0.2965779467680608 +zach brown 0.4524714828897338 +zach carson 0.6921606118546845 +zach ellison 0.6806083650190115 +zach falkner 0.25812619502868067 +zach falkner 0.2695984703632887 +zach garcia 0.30798479087452474 +zach garcia 0.3632887189292543 +zach garcia 0.7072243346007605 +zach garcia 0.7167300380228137 +zach ichabod 0.30988593155893535 +zach ichabod 0.9502868068833652 +zach king 0.5277246653919694 +zach king 0.8336520076481836 +zach king 0.9239543726235742 +zach miller 0.15678776290630975 +zach miller 0.3726235741444867 +zach miller 0.5608365019011406 +zach ovid 0.1311787072243346 +zach ovid 0.2737642585551331 +zach ovid 0.4543726235741445 +zach ovid 0.6711026615969582 +zach quirinius 0.019011406844106463 +zach robinson 0.11026615969581749 +zach steinbeck 0.28107074569789675 +zach steinbeck 0.7170172084130019 +zach thompson 0.13001912045889102 +zach thompson 0.44550669216061184 +zach underhill 0.7718631178707225 +zach white 0.7965779467680608 +zach xylophone 0.032504780114722756 +zach xylophone 0.638623326959847 +zach young 0.009560229445506692 +zach zipper 0.24334600760456274 +zach zipper 0.2832699619771863 +zach zipper 0.8087954110898662 diff --git a/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 1-3-b82dfa24123047be4b4e3c27c3997d34 b/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 1-3-b82dfa24123047be4b4e3c27c3997d34 new file mode 100644 index 0000000000000..1e0cf03db63a0 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 1-3-b82dfa24123047be4b4e3c27c3997d34 @@ -0,0 +1,1049 @@ +zach zipper 0.0 +zach zipper 0.0 +zach zipper 0.0 +zach young 0.0 +zach xylophone 0.0 +zach xylophone 0.0 +zach white 0.0 +zach underhill 0.0 +zach thompson 0.0 +zach thompson 0.0 +zach steinbeck 0.0 +zach steinbeck 0.0 +zach robinson 0.0 +zach quirinius 0.0 +zach ovid 0.0 +zach ovid 0.0 +zach ovid 0.0 +zach ovid 0.0 +zach miller 0.0 +zach miller 0.0 +zach miller 0.0 +zach king 0.0 +zach king 0.0 +zach king 0.0 +zach ichabod 0.0 +zach ichabod 0.0 +zach garcia 0.0 +zach garcia 0.0 +zach garcia 0.0 +zach garcia 0.0 +zach falkner 0.0 +zach falkner 0.0 +zach ellison 0.0 +zach carson 0.0 +zach brown 0.0 +zach brown 0.0 +zach brown 0.0 +zach brown 0.0 +zach brown 0.0 +zach allen 0.0 +yuri xylophone 0.0 +yuri white 0.0 +yuri underhill 0.0 +yuri underhill 0.0 +yuri thompson 0.0 +yuri steinbeck 0.0 +yuri steinbeck 0.0 +yuri quirinius 0.0 +yuri quirinius 0.0 +yuri quirinius 0.0 +yuri polk 0.0 +yuri polk 0.0 +yuri polk 0.0 +yuri nixon 0.0 +yuri nixon 0.0 +yuri laertes 0.0 +yuri laertes 0.0 +yuri king 0.0 +yuri johnson 0.0 +yuri johnson 0.0 +yuri johnson 0.0 +yuri hernandez 0.0 +yuri garcia 0.0 +yuri falkner 0.0 +yuri falkner 0.0 +yuri ellison 0.0 +yuri ellison 0.0 +yuri carson 0.0 +yuri carson 0.0 +yuri brown 0.0 +yuri brown 0.0 +yuri allen 0.0 +yuri allen 0.0 +xavier zipper 1.0 +xavier xylophone 0.0 +xavier white 0.0 +xavier white 0.0 +xavier underhill 0.0 +xavier thompson 0.0 +xavier quirinius 0.0 +xavier quirinius 0.0 +xavier quirinius 0.0 +xavier quirinius 0.0 +xavier polk 0.0 +xavier polk 0.0 +xavier polk 0.0 +xavier polk 0.0 +xavier ovid 0.0 +xavier laertes 0.0 +xavier king 0.0 +xavier king 0.0 +xavier johnson 0.0 +xavier johnson 0.0 +xavier ichabod 0.0 +xavier ichabod 0.0 +xavier hernandez 0.0 +xavier hernandez 0.0 +xavier hernandez 0.0 +xavier garcia 0.0 +xavier ellison 0.0 +xavier ellison 0.0 +xavier davidson 0.0 +xavier davidson 0.0 +xavier davidson 0.0 +xavier carson 1.0 +xavier carson 0.0 +xavier brown 0.0 +xavier brown 0.0 +xavier brown 0.0 +xavier allen 0.0 +xavier allen 0.0 +xavier allen 0.0 +wendy young 1.0 +wendy young 0.0 +wendy xylophone 0.0 +wendy xylophone 0.0 +wendy white 0.0 +wendy van buren 0.0 +wendy van buren 0.0 +wendy underhill 0.0 +wendy underhill 0.0 +wendy underhill 0.0 +wendy thompson 0.0 +wendy thompson 0.0 +wendy steinbeck 0.0 +wendy robinson 0.0 +wendy robinson 0.0 +wendy robinson 0.0 +wendy quirinius 0.0 +wendy quirinius 0.0 +wendy polk 0.0 +wendy polk 0.0 +wendy ovid 0.0 +wendy ovid 0.0 +wendy nixon 0.0 +wendy nixon 0.0 +wendy miller 0.0 +wendy miller 0.0 +wendy laertes 0.0 +wendy laertes 0.0 +wendy laertes 0.0 +wendy king 0.0 +wendy king 0.0 +wendy king 0.0 +wendy ichabod 0.0 +wendy hernandez 0.0 +wendy garcia 0.0 +wendy garcia 0.0 +wendy garcia 0.0 +wendy garcia 0.0 +wendy falkner 0.0 +wendy falkner 0.0 +wendy falkner 0.0 +wendy ellison 0.0 +wendy ellison 0.0 +wendy brown 0.0 +wendy brown 0.0 +wendy allen 0.0 +wendy allen 0.0 +wendy allen 0.0 +victor zipper 0.0 +victor young 0.0 +victor xylophone 0.0 +victor xylophone 0.0 +victor xylophone 0.0 +victor xylophone 0.0 +victor xylophone 0.0 +victor white 1.0 +victor white 0.0 +victor van buren 0.0 +victor van buren 0.0 +victor thompson 0.0 +victor steinbeck 0.0 +victor steinbeck 0.0 +victor steinbeck 0.0 +victor robinson 0.0 +victor robinson 0.0 +victor quirinius 0.0 +victor quirinius 0.0 +victor polk 0.0 +victor ovid 0.0 +victor nixon 0.0 +victor nixon 0.0 +victor miller 0.0 +victor laertes 0.0 +victor laertes 0.0 +victor king 0.0 +victor king 0.0 +victor johnson 0.0 +victor johnson 0.0 +victor johnson 0.0 +victor hernandez 0.0 +victor hernandez 0.0 +victor hernandez 0.0 +victor hernandez 0.0 +victor hernandez 0.0 +victor ellison 0.0 +victor ellison 0.0 +victor davidson 0.0 +victor davidson 0.0 +victor davidson 0.0 +victor brown 0.0 +victor brown 0.0 +victor brown 0.0 +victor brown 0.0 +victor allen 0.0 +victor allen 0.0 +ulysses young 0.0 +ulysses young 0.0 +ulysses young 0.0 +ulysses xylophone 0.0 +ulysses xylophone 0.0 +ulysses xylophone 0.0 +ulysses white 0.0 +ulysses white 0.0 +ulysses van buren 0.0 +ulysses underhill 0.0 +ulysses underhill 0.0 +ulysses underhill 0.0 +ulysses underhill 0.0 +ulysses underhill 0.0 +ulysses underhill 0.0 +ulysses underhill 0.0 +ulysses thompson 1.0 +ulysses steinbeck 0.0 +ulysses steinbeck 0.0 +ulysses robinson 0.0 +ulysses quirinius 0.0 +ulysses polk 1.0 +ulysses polk 0.0 +ulysses polk 0.0 +ulysses polk 0.0 +ulysses ovid 0.0 +ulysses nixon 0.0 +ulysses miller 0.0 +ulysses miller 0.0 +ulysses laertes 0.0 +ulysses laertes 0.0 +ulysses laertes 0.0 +ulysses king 0.0 +ulysses johnson 0.0 +ulysses ichabod 0.0 +ulysses ichabod 0.0 +ulysses hernandez 1.0 +ulysses hernandez 0.0 +ulysses hernandez 0.0 +ulysses garcia 0.0 +ulysses ellison 1.0 +ulysses davidson 0.0 +ulysses carson 0.0 +ulysses carson 0.0 +ulysses carson 0.0 +ulysses carson 0.0 +ulysses brown 0.0 +tom zipper 0.0 +tom young 0.0 +tom young 0.0 +tom white 0.0 +tom van buren 0.0 +tom van buren 0.0 +tom van buren 0.0 +tom steinbeck 0.0 +tom robinson 0.0 +tom robinson 0.0 +tom robinson 0.0 +tom robinson 0.0 +tom quirinius 0.0 +tom quirinius 0.0 +tom polk 0.0 +tom polk 0.0 +tom ovid 0.0 +tom nixon 0.0 +tom miller 0.0 +tom miller 0.0 +tom miller 0.0 +tom laertes 0.0 +tom laertes 0.0 +tom king 0.0 +tom johnson 0.0 +tom johnson 0.0 +tom ichabod 0.0 +tom hernandez 0.0 +tom hernandez 0.0 +tom falkner 0.0 +tom falkner 0.0 +tom ellison 0.0 +tom ellison 0.0 +tom ellison 0.0 +tom davidson 0.0 +tom carson 0.0 +tom carson 0.0 +tom carson 0.0 +tom brown 0.0 +tom brown 0.0 +sarah zipper 0.0 +sarah young 0.0 +sarah xylophone 0.0 +sarah white 0.0 +sarah white 0.0 +sarah steinbeck 0.0 +sarah robinson 0.0 +sarah robinson 0.0 +sarah ovid 0.0 +sarah miller 0.0 +sarah king 0.0 +sarah king 0.0 +sarah johnson 0.0 +sarah johnson 0.0 +sarah johnson 0.0 +sarah johnson 0.0 +sarah ichabod 0.0 +sarah ichabod 0.0 +sarah garcia 0.0 +sarah garcia 0.0 +sarah garcia 0.0 +sarah falkner 0.0 +sarah falkner 0.0 +sarah ellison 0.0 +sarah carson 0.0 +sarah carson 0.0 +sarah carson 0.0 +rachel zipper 0.0 +rachel zipper 0.0 +rachel young 0.0 +rachel white 0.0 +rachel white 0.0 +rachel underhill 0.0 +rachel thompson 0.0 +rachel thompson 0.0 +rachel thompson 0.0 +rachel robinson 1.0 +rachel robinson 0.0 +rachel robinson 0.0 +rachel quirinius 0.0 +rachel polk 0.0 +rachel ovid 0.0 +rachel ovid 0.0 +rachel laertes 0.0 +rachel laertes 0.0 +rachel king 0.0 +rachel king 0.0 +rachel johnson 0.0 +rachel falkner 0.0 +rachel falkner 0.0 +rachel falkner 0.0 +rachel falkner 0.0 +rachel ellison 0.0 +rachel davidson 0.0 +rachel carson 0.0 +rachel carson 0.0 +rachel brown 1.0 +rachel brown 0.0 +rachel brown 0.0 +rachel brown 0.0 +rachel brown 0.0 +rachel allen 0.0 +rachel allen 0.0 +quinn zipper 0.0 +quinn zipper 0.0 +quinn young 0.0 +quinn van buren 0.0 +quinn underhill 0.0 +quinn underhill 0.0 +quinn underhill 0.0 +quinn thompson 0.0 +quinn thompson 0.0 +quinn steinbeck 0.0 +quinn steinbeck 0.0 +quinn robinson 0.0 +quinn quirinius 0.0 +quinn ovid 0.0 +quinn nixon 0.0 +quinn laertes 1.0 +quinn laertes 0.0 +quinn laertes 0.0 +quinn king 1.0 +quinn king 0.0 +quinn ichabod 0.0 +quinn garcia 1.0 +quinn garcia 0.0 +quinn garcia 0.0 +quinn garcia 0.0 +quinn ellison 0.0 +quinn ellison 0.0 +quinn davidson 1.0 +quinn davidson 0.0 +quinn davidson 0.0 +quinn davidson 0.0 +quinn brown 0.0 +quinn brown 0.0 +quinn brown 0.0 +quinn allen 1.0 +quinn allen 0.0 +priscilla zipper 0.0 +priscilla zipper 0.0 +priscilla young 0.0 +priscilla young 0.0 +priscilla xylophone 0.0 +priscilla xylophone 0.0 +priscilla xylophone 0.0 +priscilla white 1.0 +priscilla van buren 0.0 +priscilla van buren 0.0 +priscilla van buren 0.0 +priscilla underhill 0.0 +priscilla underhill 0.0 +priscilla thompson 0.0 +priscilla quirinius 0.0 +priscilla polk 0.0 +priscilla ovid 0.0 +priscilla ovid 0.0 +priscilla nixon 0.0 +priscilla nixon 0.0 +priscilla king 0.0 +priscilla johnson 0.0 +priscilla johnson 0.0 +priscilla johnson 0.0 +priscilla johnson 0.0 +priscilla johnson 0.0 +priscilla ichabod 0.0 +priscilla ichabod 0.0 +priscilla carson 0.0 +priscilla carson 0.0 +priscilla carson 0.0 +priscilla brown 0.0 +priscilla brown 0.0 +priscilla brown 0.0 +oscar zipper 0.0 +oscar zipper 0.0 +oscar zipper 0.0 +oscar xylophone 0.0 +oscar xylophone 0.0 +oscar xylophone 0.0 +oscar white 0.0 +oscar white 0.0 +oscar white 0.0 +oscar white 0.0 +oscar van buren 1.0 +oscar van buren 0.0 +oscar van buren 0.0 +oscar underhill 0.0 +oscar thompson 0.0 +oscar thompson 0.0 +oscar thompson 0.0 +oscar thompson 0.0 +oscar steinbeck 0.0 +oscar robinson 0.0 +oscar robinson 0.0 +oscar robinson 0.0 +oscar robinson 0.0 +oscar quirinius 0.0 +oscar quirinius 0.0 +oscar quirinius 0.0 +oscar quirinius 0.0 +oscar polk 1.0 +oscar polk 0.0 +oscar ovid 0.0 +oscar ovid 0.0 +oscar ovid 0.0 +oscar nixon 0.0 +oscar laertes 0.0 +oscar laertes 0.0 +oscar laertes 0.0 +oscar laertes 0.0 +oscar king 0.0 +oscar king 0.0 +oscar king 0.0 +oscar johnson 0.0 +oscar johnson 0.0 +oscar ichabod 0.0 +oscar ichabod 0.0 +oscar ichabod 0.0 +oscar ichabod 0.0 +oscar hernandez 0.0 +oscar hernandez 0.0 +oscar garcia 0.0 +oscar falkner 1.0 +oscar ellison 0.0 +oscar ellison 0.0 +oscar davidson 0.0 +oscar carson 0.0 +oscar carson 0.0 +oscar carson 0.0 +oscar carson 0.0 +oscar carson 0.0 +oscar brown 0.0 +oscar allen 0.0 +nick zipper 0.0 +nick zipper 0.0 +nick young 1.0 +nick young 0.0 +nick xylophone 0.0 +nick van buren 0.0 +nick underhill 0.0 +nick thompson 0.0 +nick steinbeck 0.0 +nick robinson 0.0 +nick robinson 0.0 +nick quirinius 0.0 +nick quirinius 0.0 +nick polk 0.0 +nick ovid 0.0 +nick nixon 0.0 +nick miller 0.0 +nick laertes 0.0 +nick johnson 0.0 +nick johnson 0.0 +nick ichabod 0.0 +nick ichabod 0.0 +nick ichabod 0.0 +nick garcia 0.0 +nick garcia 0.0 +nick garcia 0.0 +nick falkner 0.0 +nick falkner 0.0 +nick ellison 0.0 +nick ellison 0.0 +nick davidson 0.0 +nick brown 0.0 +nick allen 0.0 +nick allen 0.0 +mike zipper 0.0 +mike zipper 0.0 +mike zipper 0.0 +mike young 0.0 +mike young 0.0 +mike young 0.0 +mike white 0.0 +mike white 0.0 +mike white 0.0 +mike white 0.0 +mike van buren 0.0 +mike van buren 0.0 +mike steinbeck 0.0 +mike steinbeck 0.0 +mike steinbeck 0.0 +mike steinbeck 0.0 +mike quirinius 0.0 +mike polk 0.0 +mike polk 0.0 +mike polk 0.0 +mike nixon 0.0 +mike nixon 0.0 +mike miller 0.0 +mike king 0.0 +mike king 0.0 +mike king 0.0 +mike king 0.0 +mike king 0.0 +mike king 0.0 +mike ichabod 0.0 +mike hernandez 0.0 +mike hernandez 0.0 +mike garcia 0.0 +mike garcia 0.0 +mike garcia 0.0 +mike falkner 0.0 +mike ellison 0.0 +mike ellison 0.0 +mike ellison 0.0 +mike ellison 0.0 +mike ellison 0.0 +mike davidson 0.0 +mike davidson 0.0 +mike carson 0.0 +mike carson 0.0 +mike carson 0.0 +mike brown 0.0 +mike allen 0.0 +luke zipper 0.0 +luke xylophone 0.0 +luke white 0.0 +luke van buren 0.0 +luke underhill 1.0 +luke underhill 0.0 +luke underhill 0.0 +luke thompson 0.0 +luke robinson 0.0 +luke robinson 0.0 +luke quirinius 0.0 +luke polk 0.0 +luke polk 0.0 +luke ovid 0.0 +luke ovid 0.0 +luke miller 0.0 +luke laertes 0.0 +luke laertes 0.0 +luke laertes 0.0 +luke laertes 0.0 +luke laertes 0.0 +luke johnson 0.0 +luke johnson 0.0 +luke johnson 0.0 +luke ichabod 0.0 +luke ichabod 0.0 +luke garcia 0.0 +luke garcia 0.0 +luke falkner 0.0 +luke falkner 0.0 +luke ellison 0.0 +luke ellison 0.0 +luke ellison 0.0 +luke davidson 0.0 +luke davidson 0.0 +luke brown 0.0 +luke allen 1.0 +luke allen 0.0 +luke allen 0.0 +luke allen 0.0 +luke allen 0.0 +katie zipper 1.0 +katie zipper 0.0 +katie young 1.0 +katie young 0.0 +katie young 0.0 +katie xylophone 0.0 +katie white 0.0 +katie white 0.0 +katie van buren 0.0 +katie van buren 0.0 +katie robinson 0.0 +katie polk 0.0 +katie polk 0.0 +katie ovid 0.0 +katie nixon 1.0 +katie miller 0.0 +katie miller 0.0 +katie king 0.0 +katie king 0.0 +katie king 0.0 +katie ichabod 0.0 +katie ichabod 0.0 +katie ichabod 0.0 +katie hernandez 0.0 +katie garcia 0.0 +katie garcia 0.0 +katie falkner 0.0 +katie ellison 0.0 +katie ellison 0.0 +katie davidson 0.0 +katie brown 0.0 +katie allen 0.0 +jessica zipper 0.0 +jessica zipper 0.0 +jessica zipper 0.0 +jessica young 0.0 +jessica young 0.0 +jessica xylophone 0.0 +jessica white 0.0 +jessica white 0.0 +jessica white 0.0 +jessica white 0.0 +jessica white 0.0 +jessica van buren 0.0 +jessica underhill 1.0 +jessica underhill 0.0 +jessica underhill 0.0 +jessica thompson 0.0 +jessica thompson 0.0 +jessica robinson 0.0 +jessica quirinius 1.0 +jessica quirinius 0.0 +jessica quirinius 0.0 +jessica quirinius 0.0 +jessica polk 0.0 +jessica ovid 0.0 +jessica ovid 0.0 +jessica nixon 0.0 +jessica nixon 0.0 +jessica miller 0.0 +jessica johnson 0.0 +jessica johnson 0.0 +jessica ichabod 1.0 +jessica garcia 0.0 +jessica garcia 0.0 +jessica falkner 0.0 +jessica ellison 0.0 +jessica ellison 0.0 +jessica davidson 0.0 +jessica davidson 0.0 +jessica davidson 0.0 +jessica davidson 0.0 +jessica carson 0.0 +jessica carson 0.0 +jessica carson 0.0 +jessica brown 0.0 +irene xylophone 0.0 +irene van buren 0.0 +irene van buren 0.0 +irene underhill 0.0 +irene underhill 0.0 +irene thompson 0.0 +irene steinbeck 0.0 +irene robinson 0.0 +irene quirinius 0.5 +irene quirinius 0.0 +irene quirinius 0.0 +irene polk 0.0 +irene polk 0.0 +irene polk 0.0 +irene polk 0.0 +irene polk 0.0 +irene ovid 0.0 +irene ovid 0.0 +irene ovid 0.0 +irene nixon 0.0 +irene nixon 0.0 +irene nixon 0.0 +irene miller 0.0 +irene laertes 0.0 +irene laertes 0.0 +irene laertes 0.0 +irene johnson 0.0 +irene ichabod 0.0 +irene ichabod 0.0 +irene garcia 0.0 +irene garcia 0.0 +irene garcia 0.0 +irene falkner 0.0 +irene falkner 0.0 +irene ellison 0.0 +irene ellison 0.0 +irene carson 0.0 +irene brown 0.0 +irene brown 0.0 +irene brown 0.0 +irene allen 0.0 +holly zipper 1.0 +holly zipper 0.0 +holly young 0.0 +holly young 0.0 +holly xylophone 0.0 +holly white 0.0 +holly white 0.0 +holly van buren 0.0 +holly underhill 1.0 +holly underhill 0.0 +holly underhill 0.0 +holly underhill 0.0 +holly thompson 1.0 +holly thompson 0.0 +holly thompson 0.0 +holly robinson 0.0 +holly polk 0.0 +holly polk 0.0 +holly nixon 0.0 +holly nixon 0.0 +holly miller 1.0 +holly laertes 0.0 +holly king 0.0 +holly king 0.0 +holly johnson 0.0 +holly johnson 0.0 +holly johnson 0.0 +holly ichabod 0.0 +holly ichabod 0.0 +holly ichabod 0.0 +holly hernandez 0.0 +holly hernandez 0.0 +holly hernandez 0.0 +holly hernandez 0.0 +holly falkner 0.0 +holly brown 0.0 +holly brown 0.0 +holly allen 0.0 +gabriella zipper 0.0 +gabriella zipper 0.0 +gabriella young 0.0 +gabriella young 0.0 +gabriella white 0.0 +gabriella van buren 0.0 +gabriella van buren 0.0 +gabriella thompson 0.0 +gabriella thompson 0.0 +gabriella thompson 0.0 +gabriella steinbeck 0.0 +gabriella steinbeck 0.0 +gabriella polk 0.0 +gabriella polk 0.0 +gabriella ovid 0.0 +gabriella ovid 0.0 +gabriella miller 0.0 +gabriella laertes 0.0 +gabriella king 0.0 +gabriella king 0.0 +gabriella ichabod 1.0 +gabriella ichabod 0.0 +gabriella ichabod 0.0 +gabriella ichabod 0.0 +gabriella ichabod 0.0 +gabriella hernandez 1.0 +gabriella hernandez 0.0 +gabriella garcia 0.0 +gabriella falkner 0.0 +gabriella falkner 0.0 +gabriella falkner 0.0 +gabriella ellison 0.0 +gabriella ellison 0.0 +gabriella davidson 0.0 +gabriella carson 0.0 +gabriella brown 0.0 +gabriella brown 0.0 +gabriella allen 0.0 +gabriella allen 0.0 +fred zipper 0.0 +fred young 0.0 +fred young 0.0 +fred white 0.0 +fred van buren 0.0 +fred van buren 0.0 +fred van buren 0.0 +fred van buren 0.0 +fred underhill 0.0 +fred steinbeck 0.0 +fred steinbeck 0.0 +fred steinbeck 0.0 +fred robinson 1.0 +fred quirinius 0.0 +fred quirinius 0.0 +fred polk 0.0 +fred polk 0.0 +fred polk 0.0 +fred polk 0.0 +fred nixon 0.0 +fred nixon 0.0 +fred nixon 0.0 +fred nixon 0.0 +fred miller 0.0 +fred laertes 0.0 +fred king 0.0 +fred king 0.0 +fred johnson 1.0 +fred ichabod 0.0 +fred ichabod 0.0 +fred hernandez 0.0 +fred falkner 1.0 +fred falkner 0.0 +fred falkner 0.0 +fred ellison 0.0 +fred ellison 0.0 +fred ellison 0.0 +fred davidson 0.0 +fred davidson 0.0 +fred davidson 0.0 +ethan zipper 0.0 +ethan zipper 0.0 +ethan xylophone 0.0 +ethan white 0.0 +ethan white 0.0 +ethan van buren 0.0 +ethan underhill 0.5 +ethan robinson 0.0 +ethan robinson 0.0 +ethan quirinius 0.0 +ethan quirinius 0.0 +ethan quirinius 0.0 +ethan polk 1.0 +ethan polk 0.0 +ethan polk 0.0 +ethan polk 0.0 +ethan ovid 0.0 +ethan nixon 0.0 +ethan miller 0.0 +ethan laertes 0.0 +ethan laertes 0.0 +ethan laertes 0.0 +ethan laertes 0.0 +ethan laertes 0.0 +ethan laertes 0.0 +ethan laertes 0.0 +ethan king 0.0 +ethan johnson 0.0 +ethan hernandez 0.0 +ethan garcia 0.0 +ethan falkner 0.0 +ethan falkner 0.0 +ethan ellison 0.0 +ethan ellison 0.0 +ethan carson 0.0 +ethan brown 1.0 +ethan brown 0.0 +ethan brown 0.0 +ethan brown 0.0 +ethan brown 0.0 +ethan brown 0.0 +ethan allen 0.0 +david young 0.0 +david young 0.0 +david xylophone 1.0 +david xylophone 0.0 +david xylophone 0.0 +david white 0.0 +david van buren 0.0 +david van buren 0.0 +david underhill 0.0 +david underhill 0.0 +david underhill 0.0 +david thompson 1.0 +david robinson 0.0 +david robinson 0.0 +david quirinius 0.0 +david quirinius 0.0 +david quirinius 0.0 +david ovid 0.0 +david ovid 0.0 +david nixon 0.0 +david laertes 0.0 +david ichabod 1.0 +david ichabod 0.0 +david hernandez 1.0 +david ellison 0.0 +david ellison 0.0 +david ellison 0.0 +david davidson 0.0 +david davidson 0.0 +david davidson 0.0 +david davidson 0.0 +david brown 0.0 +david brown 0.0 +david allen 0.0 +david allen 0.0 +calvin zipper 0.0 +calvin zipper 0.0 +calvin young 0.0 +calvin young 0.0 +calvin xylophone 0.0 +calvin xylophone 0.0 +calvin xylophone 0.0 +calvin white 0.0 +calvin white 0.0 +calvin van buren 1.0 +calvin van buren 0.0 +calvin underhill 0.0 +calvin thompson 0.0 +calvin thompson 0.0 +calvin steinbeck 0.0 +calvin steinbeck 0.0 +calvin steinbeck 0.0 +calvin robinson 0.0 +calvin quirinius 0.0 +calvin quirinius 0.0 +calvin polk 0.0 +calvin ovid 0.0 +calvin ovid 0.0 +calvin ovid 0.0 +calvin ovid 0.0 +calvin nixon 0.0 +calvin nixon 0.0 +calvin nixon 0.0 +calvin laertes 0.0 +calvin laertes 0.0 +calvin johnson 0.0 +calvin hernandez 0.0 +calvin garcia 0.0 +calvin falkner 0.0 +calvin falkner 0.0 +calvin falkner 0.0 +calvin falkner 0.0 +calvin falkner 0.0 +calvin falkner 0.0 +calvin ellison 0.0 +calvin davidson 0.0 +calvin davidson 0.0 +calvin carson 0.0 +calvin brown 0.0 +calvin brown 0.0 +calvin brown 0.0 +calvin allen 0.0 +bob zipper 0.0 +bob zipper 0.0 +bob zipper 0.0 +bob young 0.0 +bob xylophone 0.0 +bob xylophone 0.0 +bob white 0.0 +bob white 0.0 +bob van buren 0.0 +bob steinbeck 0.0 +bob quirinius 0.0 +bob polk 0.0 +bob ovid 0.0 +bob ovid 0.0 +bob ovid 0.0 +bob ovid 0.0 +bob miller 0.0 +bob laertes 0.0 +bob laertes 0.0 +bob king 1.0 +bob king 0.0 +bob king 0.0 +bob ichabod 0.0 +bob hernandez 1.0 +bob garcia 0.0 +bob garcia 0.0 +bob garcia 0.0 +bob garcia 0.0 +bob garcia 0.0 +bob falkner 0.0 +bob ellison 1.0 +bob ellison 0.0 +bob ellison 0.0 +bob ellison 0.0 +bob davidson 0.0 +bob davidson 0.0 +bob davidson 0.0 +bob carson 0.0 +bob brown 0.0 +bob brown 0.0 +bob brown 0.0 +alice zipper 0.0 +alice zipper 0.0 +alice zipper 0.0 +alice xylophone 0.0 +alice xylophone 0.0 +alice xylophone 0.0 +alice van buren 0.0 +alice underhill 0.0 +alice steinbeck 0.0 +alice steinbeck 0.0 +alice steinbeck 0.0 +alice robinson 0.0 +alice robinson 0.0 +alice quirinius 0.0 +alice quirinius 0.0 +alice polk 1.0 +alice ovid 0.0 +alice nixon 0.0 +alice nixon 0.0 +alice nixon 0.0 +alice miller 0.0 +alice laertes 0.0 +alice laertes 0.0 +alice king 0.0 +alice king 0.0 +alice king 0.0 +alice johnson 0.0 +alice hernandez 0.0 +alice hernandez 0.0 +alice garcia 0.0 +alice falkner 0.0 +alice davidson 0.0 +alice carson 0.0 +alice brown 0.0 +alice allen 0.0 +alice allen 0.0 +alice allen 0.0 + 0.0 + 0.0 + 0.0 diff --git a/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 2-0-81bb7f49a55385878637c8aac4d08e5 b/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 2-0-81bb7f49a55385878637c8aac4d08e5 new file mode 100644 index 0000000000000..9091a9156134c --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 2-0-81bb7f49a55385878637c8aac4d08e5 @@ -0,0 +1,1294 @@ +2013-03-01 09:11:58.70307 52.64 1 +2013-03-01 09:11:58.70307 52.64 1 +2013-03-01 09:11:58.70307 52.64 1 +2013-03-01 09:11:58.70307 52.64 1 +2013-03-01 09:11:58.703071 71.07 1 +2013-03-01 09:11:58.703071 71.07 1 +2013-03-01 09:11:58.703071 71.07 1 +2013-03-01 09:11:58.703071 71.07 1 +2013-03-01 09:11:58.703071 71.07 1 +2013-03-01 09:11:58.703071 71.07 1 +2013-03-01 09:11:58.703071 71.07 1 +2013-03-01 09:11:58.703072 2.96 1 +2013-03-01 09:11:58.703072 2.96 1 +2013-03-01 09:11:58.703072 2.96 1 +2013-03-01 09:11:58.703072 2.96 1 +2013-03-01 09:11:58.703072 2.96 1 +2013-03-01 09:11:58.703073 10.07 1 +2013-03-01 09:11:58.703073 10.07 1 +2013-03-01 09:11:58.703073 10.07 1 +2013-03-01 09:11:58.703073 10.07 1 +2013-03-01 09:11:58.703074 37.8 1 +2013-03-01 09:11:58.703074 37.8 1 +2013-03-01 09:11:58.703074 37.8 1 +2013-03-01 09:11:58.703074 37.8 1 +2013-03-01 09:11:58.703074 37.8 1 +2013-03-01 09:11:58.703074 37.8 1 +2013-03-01 09:11:58.703075 5.64 1 +2013-03-01 09:11:58.703075 5.64 1 +2013-03-01 09:11:58.703075 5.64 1 +2013-03-01 09:11:58.703075 5.64 1 +2013-03-01 09:11:58.703076 18.51 1 +2013-03-01 09:11:58.703076 18.51 1 +2013-03-01 09:11:58.703076 18.51 1 +2013-03-01 09:11:58.703076 18.51 1 +2013-03-01 09:11:58.703076 18.51 1 +2013-03-01 09:11:58.703076 18.51 1 +2013-03-01 09:11:58.703076 18.51 1 +2013-03-01 09:11:58.703077 10.16 1 +2013-03-01 09:11:58.703077 10.16 1 +2013-03-01 09:11:58.703077 10.16 1 +2013-03-01 09:11:58.703077 10.16 1 +2013-03-01 09:11:58.703077 10.16 1 +2013-03-01 09:11:58.703077 10.16 1 +2013-03-01 09:11:58.703078 61.52 1 +2013-03-01 09:11:58.703078 61.52 1 +2013-03-01 09:11:58.703078 61.52 1 +2013-03-01 09:11:58.703078 61.52 1 +2013-03-01 09:11:58.703078 61.52 1 +2013-03-01 09:11:58.703078 61.52 1 +2013-03-01 09:11:58.703079 27.32 1 +2013-03-01 09:11:58.703079 27.32 1 +2013-03-01 09:11:58.703079 27.32 1 +2013-03-01 09:11:58.703079 27.32 1 +2013-03-01 09:11:58.70308 1.76 1 +2013-03-01 09:11:58.70308 1.76 1 +2013-03-01 09:11:58.70308 1.76 1 +2013-03-01 09:11:58.70308 1.76 1 +2013-03-01 09:11:58.70308 1.76 1 +2013-03-01 09:11:58.70308 1.76 1 +2013-03-01 09:11:58.70308 1.76 1 +2013-03-01 09:11:58.70308 1.76 1 +2013-03-01 09:11:58.703081 67.9 1 +2013-03-01 09:11:58.703081 67.9 1 +2013-03-01 09:11:58.703081 67.9 1 +2013-03-01 09:11:58.703081 67.9 1 +2013-03-01 09:11:58.703081 67.9 1 +2013-03-01 09:11:58.703082 37.25 1 +2013-03-01 09:11:58.703082 37.25 1 +2013-03-01 09:11:58.703082 37.25 1 +2013-03-01 09:11:58.703082 37.25 1 +2013-03-01 09:11:58.703082 37.25 1 +2013-03-01 09:11:58.703082 37.25 1 +2013-03-01 09:11:58.703083 20.52 1 +2013-03-01 09:11:58.703083 20.52 1 +2013-03-01 09:11:58.703083 20.52 1 +2013-03-01 09:11:58.703083 20.52 1 +2013-03-01 09:11:58.703083 20.52 1 +2013-03-01 09:11:58.703083 20.52 1 +2013-03-01 09:11:58.703083 20.52 1 +2013-03-01 09:11:58.703083 20.52 1 +2013-03-01 09:11:58.703084 1.76 1 +2013-03-01 09:11:58.703084 1.76 1 +2013-03-01 09:11:58.703084 1.76 1 +2013-03-01 09:11:58.703084 1.76 1 +2013-03-01 09:11:58.703084 1.76 1 +2013-03-01 09:11:58.703085 1.01 1 +2013-03-01 09:11:58.703085 1.01 1 +2013-03-01 09:11:58.703085 1.01 1 +2013-03-01 09:11:58.703085 1.01 1 +2013-03-01 09:11:58.703085 1.01 1 +2013-03-01 09:11:58.703085 1.01 1 +2013-03-01 09:11:58.703085 1.01 1 +2013-03-01 09:11:58.703085 1.01 1 +2013-03-01 09:11:58.703086 9.96 1 +2013-03-01 09:11:58.703086 9.96 1 +2013-03-01 09:11:58.703086 9.96 1 +2013-03-01 09:11:58.703086 9.96 1 +2013-03-01 09:11:58.703086 9.96 1 +2013-03-01 09:11:58.703087 10.63 1 +2013-03-01 09:11:58.703087 10.63 1 +2013-03-01 09:11:58.703087 10.63 1 +2013-03-01 09:11:58.703087 10.63 1 +2013-03-01 09:11:58.703088 1.97 1 +2013-03-01 09:11:58.703088 1.97 1 +2013-03-01 09:11:58.703088 1.97 1 +2013-03-01 09:11:58.703088 1.97 1 +2013-03-01 09:11:58.703088 1.97 1 +2013-03-01 09:11:58.703088 1.97 1 +2013-03-01 09:11:58.703089 8.76 1 +2013-03-01 09:11:58.703089 8.76 1 +2013-03-01 09:11:58.703089 8.76 1 +2013-03-01 09:11:58.703089 8.76 1 +2013-03-01 09:11:58.70309 50.99 1 +2013-03-01 09:11:58.70309 50.99 1 +2013-03-01 09:11:58.70309 50.99 1 +2013-03-01 09:11:58.70309 50.99 1 +2013-03-01 09:11:58.70309 50.99 1 +2013-03-01 09:11:58.70309 50.99 1 +2013-03-01 09:11:58.70309 50.99 1 +2013-03-01 09:11:58.703091 15.85 1 +2013-03-01 09:11:58.703091 15.85 1 +2013-03-01 09:11:58.703091 15.85 1 +2013-03-01 09:11:58.703091 15.85 1 +2013-03-01 09:11:58.703092 36.84 1 +2013-03-01 09:11:58.703092 36.84 1 +2013-03-01 09:11:58.703092 36.84 1 +2013-03-01 09:11:58.703092 36.84 1 +2013-03-01 09:11:58.703092 36.84 1 +2013-03-01 09:11:58.703092 36.84 1 +2013-03-01 09:11:58.703093 14.85 1 +2013-03-01 09:11:58.703093 14.85 1 +2013-03-01 09:11:58.703093 14.85 1 +2013-03-01 09:11:58.703094 57.11 1 +2013-03-01 09:11:58.703094 57.11 1 +2013-03-01 09:11:58.703094 57.11 1 +2013-03-01 09:11:58.703095 9.77 1 +2013-03-01 09:11:58.703095 9.77 1 +2013-03-01 09:11:58.703095 9.77 1 +2013-03-01 09:11:58.703095 9.77 1 +2013-03-01 09:11:58.703095 9.77 1 +2013-03-01 09:11:58.703095 9.77 1 +2013-03-01 09:11:58.703095 9.77 1 +2013-03-01 09:11:58.703096 11.64 1 +2013-03-01 09:11:58.703096 11.64 1 +2013-03-01 09:11:58.703096 11.64 1 +2013-03-01 09:11:58.703096 11.64 1 +2013-03-01 09:11:58.703096 11.64 1 +2013-03-01 09:11:58.703096 11.64 1 +2013-03-01 09:11:58.703096 11.64 1 +2013-03-01 09:11:58.703096 11.64 1 +2013-03-01 09:11:58.703097 0.9 1 +2013-03-01 09:11:58.703097 0.9 1 +2013-03-01 09:11:58.703097 0.9 1 +2013-03-01 09:11:58.703098 1.35 1 +2013-03-01 09:11:58.703098 1.35 1 +2013-03-01 09:11:58.703098 1.35 1 +2013-03-01 09:11:58.703098 1.35 1 +2013-03-01 09:11:58.703098 1.35 1 +2013-03-01 09:11:58.703099 11.69 1 +2013-03-01 09:11:58.703099 11.69 1 +2013-03-01 09:11:58.703099 11.69 1 +2013-03-01 09:11:58.703101 8.72 1 +2013-03-01 09:11:58.703101 8.72 1 +2013-03-01 09:11:58.703101 8.72 1 +2013-03-01 09:11:58.703101 8.72 1 +2013-03-01 09:11:58.703101 8.72 1 +2013-03-01 09:11:58.703101 8.72 1 +2013-03-01 09:11:58.703101 8.72 1 +2013-03-01 09:11:58.703102 63.65 1 +2013-03-01 09:11:58.703102 63.65 1 +2013-03-01 09:11:58.703102 63.65 1 +2013-03-01 09:11:58.703102 63.65 1 +2013-03-01 09:11:58.703102 63.65 1 +2013-03-01 09:11:58.703102 63.65 1 +2013-03-01 09:11:58.703103 8.65 1 +2013-03-01 09:11:58.703103 8.65 1 +2013-03-01 09:11:58.703103 8.65 1 +2013-03-01 09:11:58.703103 8.65 1 +2013-03-01 09:11:58.703103 8.65 1 +2013-03-01 09:11:58.703103 8.65 1 +2013-03-01 09:11:58.703103 8.65 1 +2013-03-01 09:11:58.703104 2.04 1 +2013-03-01 09:11:58.703104 2.04 1 +2013-03-01 09:11:58.703104 2.04 1 +2013-03-01 09:11:58.703104 2.04 1 +2013-03-01 09:11:58.703104 2.04 1 +2013-03-01 09:11:58.703104 2.04 1 +2013-03-01 09:11:58.703105 28.47 1 +2013-03-01 09:11:58.703105 28.47 1 +2013-03-01 09:11:58.703106 11.81 1 +2013-03-01 09:11:58.703106 11.81 1 +2013-03-01 09:11:58.703106 11.81 1 +2013-03-01 09:11:58.703106 11.81 1 +2013-03-01 09:11:58.703107 16.26 1 +2013-03-01 09:11:58.703107 16.26 1 +2013-03-01 09:11:58.703107 16.26 1 +2013-03-01 09:11:58.703107 16.26 1 +2013-03-01 09:11:58.703107 16.26 1 +2013-03-01 09:11:58.703107 16.26 1 +2013-03-01 09:11:58.703107 16.26 1 +2013-03-01 09:11:58.703108 28.47 1 +2013-03-01 09:11:58.703108 28.47 1 +2013-03-01 09:11:58.703108 28.47 1 +2013-03-01 09:11:58.703108 28.47 1 +2013-03-01 09:11:58.703109 38.98 1 +2013-03-01 09:11:58.703109 38.98 1 +2013-03-01 09:11:58.703109 38.98 1 +2013-03-01 09:11:58.703109 38.98 1 +2013-03-01 09:11:58.70311 8.16 1 +2013-03-01 09:11:58.70311 8.16 1 +2013-03-01 09:11:58.70311 8.16 1 +2013-03-01 09:11:58.70311 8.16 1 +2013-03-01 09:11:58.703111 18.8 1 +2013-03-01 09:11:58.703111 18.8 1 +2013-03-01 09:11:58.703111 18.8 1 +2013-03-01 09:11:58.703111 18.8 1 +2013-03-01 09:11:58.703111 18.8 1 +2013-03-01 09:11:58.703111 18.8 1 +2013-03-01 09:11:58.703111 18.8 1 +2013-03-01 09:11:58.703112 13.29 1 +2013-03-01 09:11:58.703112 13.29 1 +2013-03-01 09:11:58.703112 13.29 1 +2013-03-01 09:11:58.703112 13.29 1 +2013-03-01 09:11:58.703113 21.8 1 +2013-03-01 09:11:58.703113 21.8 1 +2013-03-01 09:11:58.703113 21.8 1 +2013-03-01 09:11:58.703113 21.8 1 +2013-03-01 09:11:58.703113 21.8 1 +2013-03-01 09:11:58.703113 21.8 1 +2013-03-01 09:11:58.703113 21.8 1 +2013-03-01 09:11:58.703113 21.8 1 +2013-03-01 09:11:58.703113 21.8 1 +2013-03-01 09:11:58.703113 21.8 1 +2013-03-01 09:11:58.703114 73.94 1 +2013-03-01 09:11:58.703114 73.94 1 +2013-03-01 09:11:58.703114 73.94 1 +2013-03-01 09:11:58.703114 73.94 1 +2013-03-01 09:11:58.703114 73.94 1 +2013-03-01 09:11:58.703114 73.94 1 +2013-03-01 09:11:58.703115 27.52 1 +2013-03-01 09:11:58.703115 27.52 1 +2013-03-01 09:11:58.703115 27.52 1 +2013-03-01 09:11:58.703115 27.52 1 +2013-03-01 09:11:58.703115 27.52 1 +2013-03-01 09:11:58.703116 33.45 1 +2013-03-01 09:11:58.703116 33.45 1 +2013-03-01 09:11:58.703116 33.45 1 +2013-03-01 09:11:58.703116 33.45 1 +2013-03-01 09:11:58.703117 21.81 1 +2013-03-01 09:11:58.703117 21.81 1 +2013-03-01 09:11:58.703117 21.81 1 +2013-03-01 09:11:58.703117 21.81 1 +2013-03-01 09:11:58.703117 21.81 1 +2013-03-01 09:11:58.703117 21.81 1 +2013-03-01 09:11:58.703118 8.69 1 +2013-03-01 09:11:58.703118 8.69 1 +2013-03-01 09:11:58.703119 58.02 1 +2013-03-01 09:11:58.703119 58.02 1 +2013-03-01 09:11:58.70312 52.6 1 +2013-03-01 09:11:58.70312 52.6 1 +2013-03-01 09:11:58.70312 52.6 1 +2013-03-01 09:11:58.70312 52.6 1 +2013-03-01 09:11:58.703121 96.9 1 +2013-03-01 09:11:58.703121 96.9 1 +2013-03-01 09:11:58.703121 96.9 1 +2013-03-01 09:11:58.703121 96.9 1 +2013-03-01 09:11:58.703122 53.56 1 +2013-03-01 09:11:58.703122 53.56 1 +2013-03-01 09:11:58.703122 53.56 1 +2013-03-01 09:11:58.703122 53.56 1 +2013-03-01 09:11:58.703122 53.56 1 +2013-03-01 09:11:58.703123 94.35 1 +2013-03-01 09:11:58.703123 94.35 1 +2013-03-01 09:11:58.703123 94.35 1 +2013-03-01 09:11:58.703123 94.35 1 +2013-03-01 09:11:58.703124 8.93 1 +2013-03-01 09:11:58.703124 8.93 1 +2013-03-01 09:11:58.703124 8.93 1 +2013-03-01 09:11:58.703125 14.94 1 +2013-03-01 09:11:58.703125 14.94 1 +2013-03-01 09:11:58.703125 14.94 1 +2013-03-01 09:11:58.703126 5.49 1 +2013-03-01 09:11:58.703126 5.49 1 +2013-03-01 09:11:58.703126 5.49 1 +2013-03-01 09:11:58.703126 5.49 1 +2013-03-01 09:11:58.703127 3.98 1 +2013-03-01 09:11:58.703127 3.98 1 +2013-03-01 09:11:58.703127 3.98 1 +2013-03-01 09:11:58.703127 3.98 1 +2013-03-01 09:11:58.703128 11.45 1 +2013-03-01 09:11:58.703128 11.45 1 +2013-03-01 09:11:58.703128 11.45 1 +2013-03-01 09:11:58.703128 11.45 1 +2013-03-01 09:11:58.70313 5.83 1 +2013-03-01 09:11:58.70313 5.83 1 +2013-03-01 09:11:58.70313 5.83 1 +2013-03-01 09:11:58.70313 5.83 1 +2013-03-01 09:11:58.70313 5.83 1 +2013-03-01 09:11:58.70313 5.83 1 +2013-03-01 09:11:58.703131 1.75 1 +2013-03-01 09:11:58.703131 1.75 1 +2013-03-01 09:11:58.703131 1.75 1 +2013-03-01 09:11:58.703131 1.75 1 +2013-03-01 09:11:58.703131 1.75 1 +2013-03-01 09:11:58.703131 1.75 1 +2013-03-01 09:11:58.703131 1.75 1 +2013-03-01 09:11:58.703132 1.86 1 +2013-03-01 09:11:58.703132 1.86 1 +2013-03-01 09:11:58.703133 27.34 1 +2013-03-01 09:11:58.703133 27.34 1 +2013-03-01 09:11:58.703133 27.34 1 +2013-03-01 09:11:58.703133 27.34 1 +2013-03-01 09:11:58.703134 98.9 1 +2013-03-01 09:11:58.703134 98.9 1 +2013-03-01 09:11:58.703134 98.9 1 +2013-03-01 09:11:58.703134 98.9 1 +2013-03-01 09:11:58.703134 98.9 1 +2013-03-01 09:11:58.703135 29.14 1 +2013-03-01 09:11:58.703135 29.14 1 +2013-03-01 09:11:58.703135 29.14 1 +2013-03-01 09:11:58.703136 11.87 1 +2013-03-01 09:11:58.703136 11.87 1 +2013-03-01 09:11:58.703136 11.87 1 +2013-03-01 09:11:58.703136 11.87 1 +2013-03-01 09:11:58.703136 11.87 1 +2013-03-01 09:11:58.703136 11.87 1 +2013-03-01 09:11:58.703136 11.87 1 +2013-03-01 09:11:58.703137 18.11 1 +2013-03-01 09:11:58.703137 18.11 1 +2013-03-01 09:11:58.703137 18.11 1 +2013-03-01 09:11:58.703137 18.11 1 +2013-03-01 09:11:58.703137 18.11 1 +2013-03-01 09:11:58.703138 55.68 1 +2013-03-01 09:11:58.703138 55.68 1 +2013-03-01 09:11:58.703138 55.68 1 +2013-03-01 09:11:58.703138 55.68 1 +2013-03-01 09:11:58.703139 12.67 1 +2013-03-01 09:11:58.703139 12.67 1 +2013-03-01 09:11:58.70314 2.83 1 +2013-03-01 09:11:58.70314 2.83 1 +2013-03-01 09:11:58.70314 2.83 1 +2013-03-01 09:11:58.70314 2.83 1 +2013-03-01 09:11:58.70314 2.83 1 +2013-03-01 09:11:58.70314 2.83 1 +2013-03-01 09:11:58.70314 2.83 1 +2013-03-01 09:11:58.703141 76.06 1 +2013-03-01 09:11:58.703141 76.06 1 +2013-03-01 09:11:58.703141 76.06 1 +2013-03-01 09:11:58.703141 76.06 1 +2013-03-01 09:11:58.703141 76.06 1 +2013-03-01 09:11:58.703141 76.06 1 +2013-03-01 09:11:58.703141 76.06 1 +2013-03-01 09:11:58.703141 76.06 1 +2013-03-01 09:11:58.703142 24.25 1 +2013-03-01 09:11:58.703142 24.25 1 +2013-03-01 09:11:58.703142 24.25 1 +2013-03-01 09:11:58.703143 26.86 1 +2013-03-01 09:11:58.703143 26.86 1 +2013-03-01 09:11:58.703143 26.86 1 +2013-03-01 09:11:58.703143 26.86 1 +2013-03-01 09:11:58.703143 26.86 1 +2013-03-01 09:11:58.703143 26.86 1 +2013-03-01 09:11:58.703143 26.86 1 +2013-03-01 09:11:58.703144 3.43 1 +2013-03-01 09:11:58.703144 3.43 1 +2013-03-01 09:11:58.703144 3.43 1 +2013-03-01 09:11:58.703145 8.46 1 +2013-03-01 09:11:58.703145 8.46 1 +2013-03-01 09:11:58.703145 8.46 1 +2013-03-01 09:11:58.703145 8.46 1 +2013-03-01 09:11:58.703145 8.46 1 +2013-03-01 09:11:58.703145 8.46 1 +2013-03-01 09:11:58.703145 8.46 1 +2013-03-01 09:11:58.703145 8.46 1 +2013-03-01 09:11:58.703146 89.12 1 +2013-03-01 09:11:58.703146 89.12 1 +2013-03-01 09:11:58.703146 89.12 1 +2013-03-01 09:11:58.703146 89.12 1 +2013-03-01 09:11:58.703146 89.12 1 +2013-03-01 09:11:58.703146 89.12 1 +2013-03-01 09:11:58.703146 89.12 1 +2013-03-01 09:11:58.703146 89.12 1 +2013-03-01 09:11:58.703146 89.12 1 +2013-03-01 09:11:58.703146 89.12 1 +2013-03-01 09:11:58.703147 54.94 1 +2013-03-01 09:11:58.703147 54.94 1 +2013-03-01 09:11:58.703147 54.94 1 +2013-03-01 09:11:58.703147 54.94 1 +2013-03-01 09:11:58.703147 54.94 1 +2013-03-01 09:11:58.703148 26.97 1 +2013-03-01 09:11:58.703148 26.97 1 +2013-03-01 09:11:58.703148 26.97 1 +2013-03-01 09:11:58.703148 26.97 1 +2013-03-01 09:11:58.703148 26.97 1 +2013-03-01 09:11:58.703148 26.97 1 +2013-03-01 09:11:58.703149 58.05 1 +2013-03-01 09:11:58.703149 58.05 1 +2013-03-01 09:11:58.703149 58.05 1 +2013-03-01 09:11:58.703149 58.05 1 +2013-03-01 09:11:58.703149 58.05 1 +2013-03-01 09:11:58.703149 58.05 1 +2013-03-01 09:11:58.70315 33.01 1 +2013-03-01 09:11:58.70315 33.01 1 +2013-03-01 09:11:58.70315 33.01 1 +2013-03-01 09:11:58.70315 33.01 1 +2013-03-01 09:11:58.703151 95.69 1 +2013-03-01 09:11:58.703151 95.69 1 +2013-03-01 09:11:58.703151 95.69 1 +2013-03-01 09:11:58.703151 95.69 1 +2013-03-01 09:11:58.703151 95.69 1 +2013-03-01 09:11:58.703152 6.85 1 +2013-03-01 09:11:58.703152 6.85 1 +2013-03-01 09:11:58.703152 6.85 1 +2013-03-01 09:11:58.703152 6.85 1 +2013-03-01 09:11:58.703152 6.85 1 +2013-03-01 09:11:58.703153 4.11 1 +2013-03-01 09:11:58.703153 4.11 1 +2013-03-01 09:11:58.703153 4.11 1 +2013-03-01 09:11:58.703153 4.11 1 +2013-03-01 09:11:58.703154 16.93 1 +2013-03-01 09:11:58.703154 16.93 1 +2013-03-01 09:11:58.703154 16.93 1 +2013-03-01 09:11:58.703154 16.93 1 +2013-03-01 09:11:58.703154 16.93 1 +2013-03-01 09:11:58.703155 6.93 1 +2013-03-01 09:11:58.703155 6.93 1 +2013-03-01 09:11:58.703155 6.93 1 +2013-03-01 09:11:58.703155 6.93 1 +2013-03-01 09:11:58.703155 6.93 1 +2013-03-01 09:11:58.703155 6.93 1 +2013-03-01 09:11:58.703155 6.93 1 +2013-03-01 09:11:58.703155 6.93 1 +2013-03-01 09:11:58.703156 21.79 1 +2013-03-01 09:11:58.703156 21.79 1 +2013-03-01 09:11:58.703156 21.79 1 +2013-03-01 09:11:58.703156 21.79 1 +2013-03-01 09:11:58.703156 21.79 1 +2013-03-01 09:11:58.703156 21.79 1 +2013-03-01 09:11:58.703156 21.79 1 +2013-03-01 09:11:58.703156 21.79 1 +2013-03-01 09:11:58.703157 1.29 1 +2013-03-01 09:11:58.703157 1.29 1 +2013-03-01 09:11:58.703157 1.29 1 +2013-03-01 09:11:58.703157 1.29 1 +2013-03-01 09:11:58.703157 1.29 1 +2013-03-01 09:11:58.703157 1.29 1 +2013-03-01 09:11:58.703158 71.89 1 +2013-03-01 09:11:58.703158 71.89 1 +2013-03-01 09:11:58.703158 71.89 1 +2013-03-01 09:11:58.703159 64.06 1 +2013-03-01 09:11:58.703159 64.06 1 +2013-03-01 09:11:58.703159 64.06 1 +2013-03-01 09:11:58.703159 64.06 1 +2013-03-01 09:11:58.703159 64.06 1 +2013-03-01 09:11:58.703159 64.06 1 +2013-03-01 09:11:58.70316 11.93 1 +2013-03-01 09:11:58.70316 11.93 1 +2013-03-01 09:11:58.70316 11.93 1 +2013-03-01 09:11:58.70316 11.93 1 +2013-03-01 09:11:58.703161 15.82 1 +2013-03-01 09:11:58.703161 15.82 1 +2013-03-01 09:11:58.703161 15.82 1 +2013-03-01 09:11:58.703161 15.82 1 +2013-03-01 09:11:58.703161 15.82 1 +2013-03-01 09:11:58.703162 3.51 1 +2013-03-01 09:11:58.703162 3.51 1 +2013-03-01 09:11:58.703162 3.51 1 +2013-03-01 09:11:58.703162 3.51 1 +2013-03-01 09:11:58.703162 3.51 1 +2013-03-01 09:11:58.703163 15.7 1 +2013-03-01 09:11:58.703163 15.7 1 +2013-03-01 09:11:58.703163 15.7 1 +2013-03-01 09:11:58.703163 15.7 1 +2013-03-01 09:11:58.703163 15.7 1 +2013-03-01 09:11:58.703163 15.7 1 +2013-03-01 09:11:58.703164 30.27 1 +2013-03-01 09:11:58.703164 30.27 1 +2013-03-01 09:11:58.703164 30.27 1 +2013-03-01 09:11:58.703164 30.27 1 +2013-03-01 09:11:58.703164 30.27 1 +2013-03-01 09:11:58.703164 30.27 1 +2013-03-01 09:11:58.703165 8.38 1 +2013-03-01 09:11:58.703165 8.38 1 +2013-03-01 09:11:58.703165 8.38 1 +2013-03-01 09:11:58.703166 16.6 1 +2013-03-01 09:11:58.703166 16.6 1 +2013-03-01 09:11:58.703166 16.6 1 +2013-03-01 09:11:58.703167 17.66 1 +2013-03-01 09:11:58.703167 17.66 1 +2013-03-01 09:11:58.703167 17.66 1 +2013-03-01 09:11:58.703167 17.66 1 +2013-03-01 09:11:58.703167 17.66 1 +2013-03-01 09:11:58.703167 17.66 1 +2013-03-01 09:11:58.703168 32.03 1 +2013-03-01 09:11:58.703168 32.03 1 +2013-03-01 09:11:58.703168 32.03 1 +2013-03-01 09:11:58.703168 32.03 1 +2013-03-01 09:11:58.703168 32.03 1 +2013-03-01 09:11:58.703168 32.03 1 +2013-03-01 09:11:58.703168 32.03 1 +2013-03-01 09:11:58.703168 32.03 1 +2013-03-01 09:11:58.703169 39.96 1 +2013-03-01 09:11:58.703169 39.96 1 +2013-03-01 09:11:58.703169 39.96 1 +2013-03-01 09:11:58.703169 39.96 1 +2013-03-01 09:11:58.703169 39.96 1 +2013-03-01 09:11:58.70317 11.44 1 +2013-03-01 09:11:58.70317 11.44 1 +2013-03-01 09:11:58.70317 11.44 1 +2013-03-01 09:11:58.70317 11.44 1 +2013-03-01 09:11:58.70317 11.44 1 +2013-03-01 09:11:58.703171 24.94 1 +2013-03-01 09:11:58.703171 24.94 1 +2013-03-01 09:11:58.703171 24.94 1 +2013-03-01 09:11:58.703171 24.94 1 +2013-03-01 09:11:58.703171 24.94 1 +2013-03-01 09:11:58.703171 24.94 1 +2013-03-01 09:11:58.703172 3.44 1 +2013-03-01 09:11:58.703172 3.44 1 +2013-03-01 09:11:58.703172 3.44 1 +2013-03-01 09:11:58.703172 3.44 1 +2013-03-01 09:11:58.703172 3.44 1 +2013-03-01 09:11:58.703172 3.44 1 +2013-03-01 09:11:58.703172 3.44 1 +2013-03-01 09:11:58.703172 3.44 1 +2013-03-01 09:11:58.703173 8.77 1 +2013-03-01 09:11:58.703173 8.77 1 +2013-03-01 09:11:58.703173 8.77 1 +2013-03-01 09:11:58.703173 8.77 1 +2013-03-01 09:11:58.703174 36.33 1 +2013-03-01 09:11:58.703174 36.33 1 +2013-03-01 09:11:58.703174 36.33 1 +2013-03-01 09:11:58.703174 36.33 1 +2013-03-01 09:11:58.703175 33.37 1 +2013-03-01 09:11:58.703175 33.37 1 +2013-03-01 09:11:58.703175 33.37 1 +2013-03-01 09:11:58.703175 33.37 1 +2013-03-01 09:11:58.703175 33.37 1 +2013-03-01 09:11:58.703175 33.37 1 +2013-03-01 09:11:58.703176 28.2 1 +2013-03-01 09:11:58.703176 28.2 1 +2013-03-01 09:11:58.703176 28.2 1 +2013-03-01 09:11:58.703176 28.2 1 +2013-03-01 09:11:58.703176 28.2 1 +2013-03-01 09:11:58.703177 11.43 1 +2013-03-01 09:11:58.703177 11.43 1 +2013-03-01 09:11:58.703177 11.43 1 +2013-03-01 09:11:58.703177 11.43 1 +2013-03-01 09:11:58.703177 11.43 1 +2013-03-01 09:11:58.703177 11.43 1 +2013-03-01 09:11:58.703177 11.43 1 +2013-03-01 09:11:58.703178 9.12 1 +2013-03-01 09:11:58.703178 9.12 1 +2013-03-01 09:11:58.703178 9.12 1 +2013-03-01 09:11:58.703178 9.12 1 +2013-03-01 09:11:58.703178 9.12 1 +2013-03-01 09:11:58.703178 9.12 1 +2013-03-01 09:11:58.703178 9.12 1 +2013-03-01 09:11:58.703178 9.12 1 +2013-03-01 09:11:58.703179 10.82 1 +2013-03-01 09:11:58.703179 10.82 1 +2013-03-01 09:11:58.703179 10.82 1 +2013-03-01 09:11:58.703179 10.82 1 +2013-03-01 09:11:58.70318 10.28 1 +2013-03-01 09:11:58.70318 10.28 1 +2013-03-01 09:11:58.70318 10.28 1 +2013-03-01 09:11:58.70318 10.28 1 +2013-03-01 09:11:58.70318 10.28 1 +2013-03-01 09:11:58.70318 10.28 1 +2013-03-01 09:11:58.703181 26.6 1 +2013-03-01 09:11:58.703181 26.6 1 +2013-03-01 09:11:58.703181 26.6 1 +2013-03-01 09:11:58.703181 26.6 1 +2013-03-01 09:11:58.703181 26.6 1 +2013-03-01 09:11:58.703181 26.6 1 +2013-03-01 09:11:58.703181 26.6 1 +2013-03-01 09:11:58.703182 1.23 1 +2013-03-01 09:11:58.703182 1.23 1 +2013-03-01 09:11:58.703182 1.23 1 +2013-03-01 09:11:58.703182 1.23 1 +2013-03-01 09:11:58.703182 1.23 1 +2013-03-01 09:11:58.703183 36.74 1 +2013-03-01 09:11:58.703183 36.74 1 +2013-03-01 09:11:58.703183 36.74 1 +2013-03-01 09:11:58.703183 36.74 1 +2013-03-01 09:11:58.703184 8.95 1 +2013-03-01 09:11:58.703184 8.95 1 +2013-03-01 09:11:58.703184 8.95 1 +2013-03-01 09:11:58.703184 8.95 1 +2013-03-01 09:11:58.703184 8.95 1 +2013-03-01 09:11:58.703184 8.95 1 +2013-03-01 09:11:58.703184 8.95 1 +2013-03-01 09:11:58.703185 8.91 1 +2013-03-01 09:11:58.703185 8.91 1 +2013-03-01 09:11:58.703185 8.91 1 +2013-03-01 09:11:58.703185 8.91 1 +2013-03-01 09:11:58.703185 8.91 1 +2013-03-01 09:11:58.703186 13.81 1 +2013-03-01 09:11:58.703186 13.81 1 +2013-03-01 09:11:58.703186 13.81 1 +2013-03-01 09:11:58.703186 13.81 1 +2013-03-01 09:11:58.703187 64.89 1 +2013-03-01 09:11:58.703187 64.89 1 +2013-03-01 09:11:58.703187 64.89 1 +2013-03-01 09:11:58.703187 64.89 1 +2013-03-01 09:11:58.703187 64.89 1 +2013-03-01 09:11:58.703187 64.89 1 +2013-03-01 09:11:58.703188 32.85 1 +2013-03-01 09:11:58.703188 32.85 1 +2013-03-01 09:11:58.703188 32.85 1 +2013-03-01 09:11:58.703188 32.85 1 +2013-03-01 09:11:58.703189 36.96 1 +2013-03-01 09:11:58.703189 36.96 1 +2013-03-01 09:11:58.703189 36.96 1 +2013-03-01 09:11:58.703189 36.96 1 +2013-03-01 09:11:58.703189 36.96 1 +2013-03-01 09:11:58.703189 36.96 1 +2013-03-01 09:11:58.703189 36.96 1 +2013-03-01 09:11:58.70319 90.09 1 +2013-03-01 09:11:58.70319 90.09 1 +2013-03-01 09:11:58.70319 90.09 1 +2013-03-01 09:11:58.703192 2.63 1 +2013-03-01 09:11:58.703192 2.63 1 +2013-03-01 09:11:58.703192 2.63 1 +2013-03-01 09:11:58.703192 2.63 1 +2013-03-01 09:11:58.703193 28.42 1 +2013-03-01 09:11:58.703193 28.42 1 +2013-03-01 09:11:58.703193 28.42 1 +2013-03-01 09:11:58.703194 15.07 1 +2013-03-01 09:11:58.703194 15.07 1 +2013-03-01 09:11:58.703194 15.07 1 +2013-03-01 09:11:58.703194 15.07 1 +2013-03-01 09:11:58.703194 15.07 1 +2013-03-01 09:11:58.703194 15.07 1 +2013-03-01 09:11:58.703195 3.81 1 +2013-03-01 09:11:58.703195 3.81 1 +2013-03-01 09:11:58.703195 3.81 1 +2013-03-01 09:11:58.703195 3.81 1 +2013-03-01 09:11:58.703195 3.81 1 +2013-03-01 09:11:58.703195 3.81 1 +2013-03-01 09:11:58.703196 0.08 1 +2013-03-01 09:11:58.703196 0.08 1 +2013-03-01 09:11:58.703196 0.08 1 +2013-03-01 09:11:58.703197 16.01 1 +2013-03-01 09:11:58.703197 16.01 1 +2013-03-01 09:11:58.703197 16.01 1 +2013-03-01 09:11:58.703197 16.01 1 +2013-03-01 09:11:58.703197 16.01 1 +2013-03-01 09:11:58.703197 16.01 1 +2013-03-01 09:11:58.703198 30.6 1 +2013-03-01 09:11:58.703198 30.6 1 +2013-03-01 09:11:58.703198 30.6 1 +2013-03-01 09:11:58.703198 30.6 1 +2013-03-01 09:11:58.703199 45.69 1 +2013-03-01 09:11:58.703199 45.69 1 +2013-03-01 09:11:58.703199 45.69 1 +2013-03-01 09:11:58.703199 45.69 1 +2013-03-01 09:11:58.7032 12.72 1 +2013-03-01 09:11:58.7032 12.72 1 +2013-03-01 09:11:58.703201 35.15 1 +2013-03-01 09:11:58.703201 35.15 1 +2013-03-01 09:11:58.703202 31.41 1 +2013-03-01 09:11:58.703202 31.41 1 +2013-03-01 09:11:58.703202 31.41 1 +2013-03-01 09:11:58.703202 31.41 1 +2013-03-01 09:11:58.703202 31.41 1 +2013-03-01 09:11:58.703203 11.63 1 +2013-03-01 09:11:58.703203 11.63 1 +2013-03-01 09:11:58.703203 11.63 1 +2013-03-01 09:11:58.703203 11.63 1 +2013-03-01 09:11:58.703203 11.63 1 +2013-03-01 09:11:58.703205 35.8 1 +2013-03-01 09:11:58.703205 35.8 1 +2013-03-01 09:11:58.703205 35.8 1 +2013-03-01 09:11:58.703205 35.8 1 +2013-03-01 09:11:58.703205 35.8 1 +2013-03-01 09:11:58.703206 6.61 1 +2013-03-01 09:11:58.703206 6.61 1 +2013-03-01 09:11:58.703206 6.61 1 +2013-03-01 09:11:58.703206 6.61 1 +2013-03-01 09:11:58.703207 21.14 1 +2013-03-01 09:11:58.703207 21.14 1 +2013-03-01 09:11:58.703207 21.14 1 +2013-03-01 09:11:58.703207 21.14 1 +2013-03-01 09:11:58.703207 21.14 1 +2013-03-01 09:11:58.703207 21.14 1 +2013-03-01 09:11:58.703207 21.14 1 +2013-03-01 09:11:58.703207 21.14 1 +2013-03-01 09:11:58.703208 1.23 1 +2013-03-01 09:11:58.703208 1.23 1 +2013-03-01 09:11:58.703208 1.23 1 +2013-03-01 09:11:58.703208 1.23 1 +2013-03-01 09:11:58.703208 1.23 1 +2013-03-01 09:11:58.703208 1.23 1 +2013-03-01 09:11:58.703208 1.23 1 +2013-03-01 09:11:58.703208 1.23 1 +2013-03-01 09:11:58.703209 25.92 1 +2013-03-01 09:11:58.703209 25.92 1 +2013-03-01 09:11:58.70321 37.12 1 +2013-03-01 09:11:58.70321 37.12 1 +2013-03-01 09:11:58.70321 37.12 1 +2013-03-01 09:11:58.70321 37.12 1 +2013-03-01 09:11:58.70321 37.12 1 +2013-03-01 09:11:58.70321 37.12 1 +2013-03-01 09:11:58.70321 37.12 1 +2013-03-01 09:11:58.70321 37.12 1 +2013-03-01 09:11:58.703211 5.24 1 +2013-03-01 09:11:58.703211 5.24 1 +2013-03-01 09:11:58.703211 5.24 1 +2013-03-01 09:11:58.703211 5.24 1 +2013-03-01 09:11:58.703211 5.24 1 +2013-03-01 09:11:58.703211 5.24 1 +2013-03-01 09:11:58.703212 10.52 1 +2013-03-01 09:11:58.703212 10.52 1 +2013-03-01 09:11:58.703212 10.52 1 +2013-03-01 09:11:58.703212 10.52 1 +2013-03-01 09:11:58.703212 10.52 1 +2013-03-01 09:11:58.703212 10.52 1 +2013-03-01 09:11:58.703213 38.71 1 +2013-03-01 09:11:58.703213 38.71 1 +2013-03-01 09:11:58.703213 38.71 1 +2013-03-01 09:11:58.703213 38.71 1 +2013-03-01 09:11:58.703214 31.35 1 +2013-03-01 09:11:58.703214 31.35 1 +2013-03-01 09:11:58.703214 31.35 1 +2013-03-01 09:11:58.703215 18.78 1 +2013-03-01 09:11:58.703215 18.78 1 +2013-03-01 09:11:58.703215 18.78 1 +2013-03-01 09:11:58.703215 18.78 1 +2013-03-01 09:11:58.703216 11.23 1 +2013-03-01 09:11:58.703216 11.23 1 +2013-03-01 09:11:58.703216 11.23 1 +2013-03-01 09:11:58.703216 11.23 1 +2013-03-01 09:11:58.703216 11.23 1 +2013-03-01 09:11:58.703216 11.23 1 +2013-03-01 09:11:58.703216 11.23 1 +2013-03-01 09:11:58.703216 11.23 1 +2013-03-01 09:11:58.703217 23.57 1 +2013-03-01 09:11:58.703217 23.57 1 +2013-03-01 09:11:58.703217 23.57 1 +2013-03-01 09:11:58.703218 9.67 1 +2013-03-01 09:11:58.703218 9.67 1 +2013-03-01 09:11:58.703218 9.67 1 +2013-03-01 09:11:58.703218 9.67 1 +2013-03-01 09:11:58.703218 9.67 1 +2013-03-01 09:11:58.703219 1.42 1 +2013-03-01 09:11:58.703219 1.42 1 +2013-03-01 09:11:58.703219 1.42 1 +2013-03-01 09:11:58.703219 1.42 1 +2013-03-01 09:11:58.703219 1.42 1 +2013-03-01 09:11:58.703219 1.42 1 +2013-03-01 09:11:58.70322 7.37 1 +2013-03-01 09:11:58.70322 7.37 1 +2013-03-01 09:11:58.70322 7.37 1 +2013-03-01 09:11:58.703221 26.64 1 +2013-03-01 09:11:58.703221 26.64 1 +2013-03-01 09:11:58.703221 26.64 1 +2013-03-01 09:11:58.703221 26.64 1 +2013-03-01 09:11:58.703221 26.64 1 +2013-03-01 09:11:58.703221 26.64 1 +2013-03-01 09:11:58.703221 26.64 1 +2013-03-01 09:11:58.703222 38.23 1 +2013-03-01 09:11:58.703222 38.23 1 +2013-03-01 09:11:58.703222 38.23 1 +2013-03-01 09:11:58.703222 38.23 1 +2013-03-01 09:11:58.703222 38.23 1 +2013-03-01 09:11:58.703222 38.23 1 +2013-03-01 09:11:58.703223 3.43 1 +2013-03-01 09:11:58.703223 3.43 1 +2013-03-01 09:11:58.703223 3.43 1 +2013-03-01 09:11:58.703223 3.43 1 +2013-03-01 09:11:58.703223 3.43 1 +2013-03-01 09:11:58.703224 17.92 1 +2013-03-01 09:11:58.703224 17.92 1 +2013-03-01 09:11:58.703224 17.92 1 +2013-03-01 09:11:58.703224 17.92 1 +2013-03-01 09:11:58.703224 17.92 1 +2013-03-01 09:11:58.703224 17.92 1 +2013-03-01 09:11:58.703225 35.51 1 +2013-03-01 09:11:58.703225 35.51 1 +2013-03-01 09:11:58.703225 35.51 1 +2013-03-01 09:11:58.703225 35.51 1 +2013-03-01 09:11:58.703226 34.31 1 +2013-03-01 09:11:58.703226 34.31 1 +2013-03-01 09:11:58.703226 34.31 1 +2013-03-01 09:11:58.703226 34.31 1 +2013-03-01 09:11:58.703226 34.31 1 +2013-03-01 09:11:58.703226 34.31 1 +2013-03-01 09:11:58.703226 34.31 1 +2013-03-01 09:11:58.703227 17.65 1 +2013-03-01 09:11:58.703227 17.65 1 +2013-03-01 09:11:58.703227 17.65 1 +2013-03-01 09:11:58.703227 17.65 1 +2013-03-01 09:11:58.703227 17.65 1 +2013-03-01 09:11:58.703228 4.19 1 +2013-03-01 09:11:58.703228 4.19 1 +2013-03-01 09:11:58.703229 88.52 1 +2013-03-01 09:11:58.703229 88.52 1 +2013-03-01 09:11:58.703229 88.52 1 +2013-03-01 09:11:58.703229 88.52 1 +2013-03-01 09:11:58.703229 88.52 1 +2013-03-01 09:11:58.703229 88.52 1 +2013-03-01 09:11:58.703229 88.52 1 +2013-03-01 09:11:58.703229 88.52 1 +2013-03-01 09:11:58.703229 88.52 1 +2013-03-01 09:11:58.703229 88.52 1 +2013-03-01 09:11:58.70323 28.02 1 +2013-03-01 09:11:58.70323 28.02 1 +2013-03-01 09:11:58.70323 28.02 1 +2013-03-01 09:11:58.70323 28.02 1 +2013-03-01 09:11:58.70323 28.02 1 +2013-03-01 09:11:58.703231 11.99 1 +2013-03-01 09:11:58.703231 11.99 1 +2013-03-01 09:11:58.703231 11.99 1 +2013-03-01 09:11:58.703231 11.99 1 +2013-03-01 09:11:58.703231 11.99 1 +2013-03-01 09:11:58.703231 11.99 1 +2013-03-01 09:11:58.703232 61.96 1 +2013-03-01 09:11:58.703232 61.96 1 +2013-03-01 09:11:58.703232 61.96 1 +2013-03-01 09:11:58.703233 40.81 1 +2013-03-01 09:11:58.703233 40.81 1 +2013-03-01 09:11:58.703233 40.81 1 +2013-03-01 09:11:58.703233 40.81 1 +2013-03-01 09:11:58.703233 40.81 1 +2013-03-01 09:11:58.703233 40.81 1 +2013-03-01 09:11:58.703234 44.1 1 +2013-03-01 09:11:58.703234 44.1 1 +2013-03-01 09:11:58.703234 44.1 1 +2013-03-01 09:11:58.703235 6.35 1 +2013-03-01 09:11:58.703235 6.35 1 +2013-03-01 09:11:58.703235 6.35 1 +2013-03-01 09:11:58.703235 6.35 1 +2013-03-01 09:11:58.703235 6.35 1 +2013-03-01 09:11:58.703235 6.35 1 +2013-03-01 09:11:58.703235 6.35 1 +2013-03-01 09:11:58.703236 37.8 1 +2013-03-01 09:11:58.703236 37.8 1 +2013-03-01 09:11:58.703236 37.8 1 +2013-03-01 09:11:58.703236 37.8 1 +2013-03-01 09:11:58.703236 37.8 1 +2013-03-01 09:11:58.703237 0.24 1 +2013-03-01 09:11:58.703237 0.24 1 +2013-03-01 09:11:58.703237 0.24 1 +2013-03-01 09:11:58.703237 0.24 1 +2013-03-01 09:11:58.703237 0.24 1 +2013-03-01 09:11:58.703237 0.24 1 +2013-03-01 09:11:58.703237 0.24 1 +2013-03-01 09:11:58.703237 0.24 1 +2013-03-01 09:11:58.703238 6 1 +2013-03-01 09:11:58.703238 6 1 +2013-03-01 09:11:58.703238 6 1 +2013-03-01 09:11:58.703238 6 1 +2013-03-01 09:11:58.703239 24.8 1 +2013-03-01 09:11:58.703239 24.8 1 +2013-03-01 09:11:58.703239 24.8 1 +2013-03-01 09:11:58.703239 24.8 1 +2013-03-01 09:11:58.703239 24.8 1 +2013-03-01 09:11:58.70324 5.1 1 +2013-03-01 09:11:58.70324 5.1 1 +2013-03-01 09:11:58.703241 19.33 1 +2013-03-01 09:11:58.703241 19.33 1 +2013-03-01 09:11:58.703241 19.33 1 +2013-03-01 09:11:58.703241 19.33 1 +2013-03-01 09:11:58.703241 19.33 1 +2013-03-01 09:11:58.703241 19.33 1 +2013-03-01 09:11:58.703241 19.33 1 +2013-03-01 09:11:58.703242 31.23 1 +2013-03-01 09:11:58.703242 31.23 1 +2013-03-01 09:11:58.703242 31.23 1 +2013-03-01 09:11:58.703242 31.23 1 +2013-03-01 09:11:58.703243 6.15 1 +2013-03-01 09:11:58.703243 6.15 1 +2013-03-01 09:11:58.703243 6.15 1 +2013-03-01 09:11:58.703243 6.15 1 +2013-03-01 09:11:58.703244 4.47 1 +2013-03-01 09:11:58.703244 4.47 1 +2013-03-01 09:11:58.703244 4.47 1 +2013-03-01 09:11:58.703245 0.72 1 +2013-03-01 09:11:58.703245 0.72 1 +2013-03-01 09:11:58.703245 0.72 1 +2013-03-01 09:11:58.703245 0.72 1 +2013-03-01 09:11:58.703246 45.94 1 +2013-03-01 09:11:58.703246 45.94 1 +2013-03-01 09:11:58.703247 1.29 1 +2013-03-01 09:11:58.703247 1.29 1 +2013-03-01 09:11:58.703247 1.29 1 +2013-03-01 09:11:58.703247 1.29 1 +2013-03-01 09:11:58.703247 1.29 1 +2013-03-01 09:11:58.703247 1.29 1 +2013-03-01 09:11:58.703248 14.36 1 +2013-03-01 09:11:58.703248 14.36 1 +2013-03-01 09:11:58.703248 14.36 1 +2013-03-01 09:11:58.703248 14.36 1 +2013-03-01 09:11:58.703248 14.36 1 +2013-03-01 09:11:58.703248 14.36 1 +2013-03-01 09:11:58.703248 14.36 1 +2013-03-01 09:11:58.703248 14.36 1 +2013-03-01 09:11:58.703249 19.42 1 +2013-03-01 09:11:58.703249 19.42 1 +2013-03-01 09:11:58.70325 25.89 1 +2013-03-01 09:11:58.70325 25.89 1 +2013-03-01 09:11:58.70325 25.89 1 +2013-03-01 09:11:58.70325 25.89 1 +2013-03-01 09:11:58.70325 25.89 1 +2013-03-01 09:11:58.70325 25.89 1 +2013-03-01 09:11:58.703251 68.98 1 +2013-03-01 09:11:58.703251 68.98 1 +2013-03-01 09:11:58.703251 68.98 1 +2013-03-01 09:11:58.703252 49.85 1 +2013-03-01 09:11:58.703252 49.85 1 +2013-03-01 09:11:58.703252 49.85 1 +2013-03-01 09:11:58.703252 49.85 1 +2013-03-01 09:11:58.703252 49.85 1 +2013-03-01 09:11:58.703252 49.85 1 +2013-03-01 09:11:58.703252 49.85 1 +2013-03-01 09:11:58.703253 55.75 1 +2013-03-01 09:11:58.703253 55.75 1 +2013-03-01 09:11:58.703253 55.75 1 +2013-03-01 09:11:58.703253 55.75 1 +2013-03-01 09:11:58.703253 55.75 1 +2013-03-01 09:11:58.703254 0.32 1 +2013-03-01 09:11:58.703254 0.32 1 +2013-03-01 09:11:58.703254 0.32 1 +2013-03-01 09:11:58.703254 0.32 1 +2013-03-01 09:11:58.703255 5.42 1 +2013-03-01 09:11:58.703255 5.42 1 +2013-03-01 09:11:58.703255 5.42 1 +2013-03-01 09:11:58.703255 5.42 1 +2013-03-01 09:11:58.703255 5.42 1 +2013-03-01 09:11:58.703255 5.42 1 +2013-03-01 09:11:58.703256 23.78 1 +2013-03-01 09:11:58.703256 23.78 1 +2013-03-01 09:11:58.703256 23.78 1 +2013-03-01 09:11:58.703257 14.96 1 +2013-03-01 09:11:58.703257 14.96 1 +2013-03-01 09:11:58.703257 14.96 1 +2013-03-01 09:11:58.703257 14.96 1 +2013-03-01 09:11:58.703257 14.96 1 +2013-03-01 09:11:58.703257 14.96 1 +2013-03-01 09:11:58.703257 14.96 1 +2013-03-01 09:11:58.703257 14.96 1 +2013-03-01 09:11:58.703258 19.65 1 +2013-03-01 09:11:58.703258 19.65 1 +2013-03-01 09:11:58.703258 19.65 1 +2013-03-01 09:11:58.703258 19.65 1 +2013-03-01 09:11:58.703258 19.65 1 +2013-03-01 09:11:58.703258 19.65 1 +2013-03-01 09:11:58.703259 11.37 1 +2013-03-01 09:11:58.703259 11.37 1 +2013-03-01 09:11:58.703259 11.37 1 +2013-03-01 09:11:58.703259 11.37 1 +2013-03-01 09:11:58.703259 11.37 1 +2013-03-01 09:11:58.703259 11.37 1 +2013-03-01 09:11:58.703259 11.37 1 +2013-03-01 09:11:58.70326 3.81 1 +2013-03-01 09:11:58.70326 3.81 1 +2013-03-01 09:11:58.70326 3.81 1 +2013-03-01 09:11:58.70326 3.81 1 +2013-03-01 09:11:58.703261 8.66 1 +2013-03-01 09:11:58.703261 8.66 1 +2013-03-01 09:11:58.703261 8.66 1 +2013-03-01 09:11:58.703261 8.66 1 +2013-03-01 09:11:58.703261 8.66 1 +2013-03-01 09:11:58.703261 8.66 1 +2013-03-01 09:11:58.703261 8.66 1 +2013-03-01 09:11:58.703262 1.81 1 +2013-03-01 09:11:58.703262 1.81 1 +2013-03-01 09:11:58.703262 1.81 1 +2013-03-01 09:11:58.703262 1.81 1 +2013-03-01 09:11:58.703262 1.81 1 +2013-03-01 09:11:58.703262 1.81 1 +2013-03-01 09:11:58.703262 1.81 1 +2013-03-01 09:11:58.703262 1.81 1 +2013-03-01 09:11:58.703262 1.81 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703264 52.49 1 +2013-03-01 09:11:58.703264 52.49 1 +2013-03-01 09:11:58.703264 52.49 1 +2013-03-01 09:11:58.703264 52.49 1 +2013-03-01 09:11:58.703265 6.88 1 +2013-03-01 09:11:58.703265 6.88 1 +2013-03-01 09:11:58.703265 6.88 1 +2013-03-01 09:11:58.703265 6.88 1 +2013-03-01 09:11:58.703265 6.88 1 +2013-03-01 09:11:58.703265 6.88 1 +2013-03-01 09:11:58.703265 6.88 1 +2013-03-01 09:11:58.703266 47.71 1 +2013-03-01 09:11:58.703266 47.71 1 +2013-03-01 09:11:58.703266 47.71 1 +2013-03-01 09:11:58.703266 47.71 1 +2013-03-01 09:11:58.703267 12.22 1 +2013-03-01 09:11:58.703267 12.22 1 +2013-03-01 09:11:58.703267 12.22 1 +2013-03-01 09:11:58.703267 12.22 1 +2013-03-01 09:11:58.703267 12.22 1 +2013-03-01 09:11:58.703267 12.22 1 +2013-03-01 09:11:58.703268 94.73 1 +2013-03-01 09:11:58.703268 94.73 1 +2013-03-01 09:11:58.703268 94.73 1 +2013-03-01 09:11:58.703268 94.73 1 +2013-03-01 09:11:58.703268 94.73 1 +2013-03-01 09:11:58.703268 94.73 1 +2013-03-01 09:11:58.703268 94.73 1 +2013-03-01 09:11:58.703268 94.73 1 +2013-03-01 09:11:58.703269 43.84 1 +2013-03-01 09:11:58.703269 43.84 1 +2013-03-01 09:11:58.703269 43.84 1 +2013-03-01 09:11:58.70327 5.01 1 +2013-03-01 09:11:58.70327 5.01 1 +2013-03-01 09:11:58.70327 5.01 1 +2013-03-01 09:11:58.70327 5.01 1 +2013-03-01 09:11:58.703271 61.16 1 +2013-03-01 09:11:58.703271 61.16 1 +2013-03-01 09:11:58.703272 7.49 1 +2013-03-01 09:11:58.703272 7.49 1 +2013-03-01 09:11:58.703272 7.49 1 +2013-03-01 09:11:58.703272 7.49 1 +2013-03-01 09:11:58.703273 10.94 1 +2013-03-01 09:11:58.703274 10.79 1 +2013-03-01 09:11:58.703274 10.79 1 +2013-03-01 09:11:58.703274 10.79 1 +2013-03-01 09:11:58.703274 10.79 1 +2013-03-01 09:11:58.703274 10.79 1 +2013-03-01 09:11:58.703274 10.79 1 +2013-03-01 09:11:58.703274 10.79 1 +2013-03-01 09:11:58.703275 20.57 1 +2013-03-01 09:11:58.703275 20.57 1 +2013-03-01 09:11:58.703275 20.57 1 +2013-03-01 09:11:58.703276 63.54 1 +2013-03-01 09:11:58.703276 63.54 1 +2013-03-01 09:11:58.703276 63.54 1 +2013-03-01 09:11:58.703276 63.54 1 +2013-03-01 09:11:58.703277 3.37 1 +2013-03-01 09:11:58.703277 3.37 1 +2013-03-01 09:11:58.703277 3.37 1 +2013-03-01 09:11:58.703277 3.37 1 +2013-03-01 09:11:58.703278 9.74 1 +2013-03-01 09:11:58.703278 9.74 1 +2013-03-01 09:11:58.703278 9.74 1 +2013-03-01 09:11:58.703278 9.74 1 +2013-03-01 09:11:58.703278 9.74 1 +2013-03-01 09:11:58.703278 9.74 1 +2013-03-01 09:11:58.703279 20.85 1 +2013-03-01 09:11:58.703279 20.85 1 +2013-03-01 09:11:58.703279 20.85 1 +2013-03-01 09:11:58.703279 20.85 1 +2013-03-01 09:11:58.703279 20.85 1 +2013-03-01 09:11:58.703279 20.85 1 +2013-03-01 09:11:58.703279 20.85 1 +2013-03-01 09:11:58.70328 40.68 1 +2013-03-01 09:11:58.70328 40.68 1 +2013-03-01 09:11:58.70328 40.68 1 +2013-03-01 09:11:58.70328 40.68 1 +2013-03-01 09:11:58.70328 40.68 1 +2013-03-01 09:11:58.70328 40.68 1 +2013-03-01 09:11:58.70328 40.68 1 +2013-03-01 09:11:58.703281 19.95 1 +2013-03-01 09:11:58.703281 19.95 1 +2013-03-01 09:11:58.703281 19.95 1 +2013-03-01 09:11:58.703281 19.95 1 +2013-03-01 09:11:58.703282 7.5 1 +2013-03-01 09:11:58.703282 7.5 1 +2013-03-01 09:11:58.703282 7.5 1 +2013-03-01 09:11:58.703282 7.5 1 +2013-03-01 09:11:58.703282 7.5 1 +2013-03-01 09:11:58.703282 7.5 1 +2013-03-01 09:11:58.703283 17.62 1 +2013-03-01 09:11:58.703283 17.62 1 +2013-03-01 09:11:58.703283 17.62 1 +2013-03-01 09:11:58.703283 17.62 1 +2013-03-01 09:11:58.703283 17.62 1 +2013-03-01 09:11:58.703283 17.62 1 +2013-03-01 09:11:58.703283 17.62 1 +2013-03-01 09:11:58.703283 17.62 1 +2013-03-01 09:11:58.703284 99.88 1 +2013-03-01 09:11:58.703284 99.88 1 +2013-03-01 09:11:58.703284 99.88 1 +2013-03-01 09:11:58.703284 99.88 1 +2013-03-01 09:11:58.703284 99.88 1 +2013-03-01 09:11:58.703284 99.88 1 +2013-03-01 09:11:58.703285 58.66 1 +2013-03-01 09:11:58.703285 58.66 1 +2013-03-01 09:11:58.703285 58.66 1 +2013-03-01 09:11:58.703285 58.66 1 +2013-03-01 09:11:58.703285 58.66 1 +2013-03-01 09:11:58.703286 9.53 1 +2013-03-01 09:11:58.703286 9.53 1 +2013-03-01 09:11:58.703286 9.53 1 +2013-03-01 09:11:58.703286 9.53 1 +2013-03-01 09:11:58.703287 0.89 1 +2013-03-01 09:11:58.703287 0.89 1 +2013-03-01 09:11:58.703287 0.89 1 +2013-03-01 09:11:58.703288 60.57 1 +2013-03-01 09:11:58.703288 60.57 1 +2013-03-01 09:11:58.703289 0.56 1 +2013-03-01 09:11:58.703289 0.56 1 +2013-03-01 09:11:58.703289 0.56 1 +2013-03-01 09:11:58.703289 0.56 1 +2013-03-01 09:11:58.703289 0.56 1 +2013-03-01 09:11:58.703289 0.56 1 +2013-03-01 09:11:58.703289 0.56 1 +2013-03-01 09:11:58.70329 16.89 1 +2013-03-01 09:11:58.70329 16.89 1 +2013-03-01 09:11:58.70329 16.89 1 +2013-03-01 09:11:58.70329 16.89 1 +2013-03-01 09:11:58.70329 16.89 1 +2013-03-01 09:11:58.70329 16.89 1 +2013-03-01 09:11:58.703291 1.15 1 +2013-03-01 09:11:58.703291 1.15 1 +2013-03-01 09:11:58.703291 1.15 1 +2013-03-01 09:11:58.703291 1.15 1 +2013-03-01 09:11:58.703292 4.24 1 +2013-03-01 09:11:58.703292 4.24 1 +2013-03-01 09:11:58.703292 4.24 1 +2013-03-01 09:11:58.703292 4.24 1 +2013-03-01 09:11:58.703293 42.86 1 +2013-03-01 09:11:58.703293 42.86 1 +2013-03-01 09:11:58.703293 42.86 1 +2013-03-01 09:11:58.703294 29.74 1 +2013-03-01 09:11:58.703294 29.74 1 +2013-03-01 09:11:58.703294 29.74 1 +2013-03-01 09:11:58.703294 29.74 1 +2013-03-01 09:11:58.703294 29.74 1 +2013-03-01 09:11:58.703294 29.74 1 +2013-03-01 09:11:58.703295 8.58 1 +2013-03-01 09:11:58.703295 8.58 1 +2013-03-01 09:11:58.703295 8.58 1 +2013-03-01 09:11:58.703295 8.58 1 +2013-03-01 09:11:58.703295 8.58 1 +2013-03-01 09:11:58.703295 8.58 1 +2013-03-01 09:11:58.703295 8.58 1 +2013-03-01 09:11:58.703295 8.58 1 +2013-03-01 09:11:58.703296 30.43 1 +2013-03-01 09:11:58.703296 30.43 1 +2013-03-01 09:11:58.703296 30.43 1 +2013-03-01 09:11:58.703296 30.43 1 +2013-03-01 09:11:58.703296 30.43 1 +2013-03-01 09:11:58.703296 30.43 1 +2013-03-01 09:11:58.703296 30.43 1 +2013-03-01 09:11:58.703297 25.67 1 +2013-03-01 09:11:58.703297 25.67 1 +2013-03-01 09:11:58.703297 25.67 1 +2013-03-01 09:11:58.703297 25.67 1 +2013-03-01 09:11:58.703297 25.67 1 +2013-03-01 09:11:58.703297 25.67 1 +2013-03-01 09:11:58.703298 8.8 1 +2013-03-01 09:11:58.703298 8.8 1 +2013-03-01 09:11:58.703298 8.8 1 +2013-03-01 09:11:58.703298 8.8 1 +2013-03-01 09:11:58.703299 9 1 +2013-03-01 09:11:58.703299 9 1 +2013-03-01 09:11:58.7033 7.51 1 +2013-03-01 09:11:58.7033 7.51 1 +2013-03-01 09:11:58.7033 7.51 1 +2013-03-01 09:11:58.7033 7.51 1 +2013-03-01 09:11:58.7033 7.51 1 +2013-03-01 09:11:58.703301 39.32 1 +2013-03-01 09:11:58.703301 39.32 1 +2013-03-01 09:11:58.703301 39.32 1 +2013-03-01 09:11:58.703301 39.32 1 +2013-03-01 09:11:58.703301 39.32 1 +2013-03-01 09:11:58.703301 39.32 1 +2013-03-01 09:11:58.703302 23.55 1 +2013-03-01 09:11:58.703302 23.55 1 +2013-03-01 09:11:58.703302 23.55 1 +2013-03-01 09:11:58.703302 23.55 1 +2013-03-01 09:11:58.703302 23.55 1 +2013-03-01 09:11:58.703303 88.64 1 +2013-03-01 09:11:58.703303 88.64 1 +2013-03-01 09:11:58.703303 88.64 1 +2013-03-01 09:11:58.703303 88.64 1 +2013-03-01 09:11:58.703303 88.64 1 +2013-03-01 09:11:58.703304 9.04 1 +2013-03-01 09:11:58.703304 9.04 1 +2013-03-01 09:11:58.703304 9.04 1 +2013-03-01 09:11:58.703304 9.04 1 +2013-03-01 09:11:58.703305 18.68 1 +2013-03-01 09:11:58.703305 18.68 1 +2013-03-01 09:11:58.703305 18.68 1 +2013-03-01 09:11:58.703305 18.68 1 +2013-03-01 09:11:58.703306 3.95 1 +2013-03-01 09:11:58.703306 3.95 1 +2013-03-01 09:11:58.703306 3.95 1 +2013-03-01 09:11:58.703306 3.95 1 +2013-03-01 09:11:58.703307 31.28 1 +2013-03-01 09:11:58.703307 31.28 1 +2013-03-01 09:11:58.703307 31.28 1 +2013-03-01 09:11:58.703308 16.95 1 +2013-03-01 09:11:58.703308 16.95 1 +2013-03-01 09:11:58.703308 16.95 1 +2013-03-01 09:11:58.703308 16.95 1 +2013-03-01 09:11:58.703309 11.16 1 +2013-03-01 09:11:58.703309 11.16 1 +2013-03-01 09:11:58.703309 11.16 1 +2013-03-01 09:11:58.703309 11.16 1 +2013-03-01 09:11:58.70331 9.24 1 +2013-03-01 09:11:58.70331 9.24 1 +2013-03-01 09:11:58.70331 9.24 1 +2013-03-01 09:11:58.70331 9.24 1 +2013-03-01 09:11:58.70331 9.24 1 +2013-03-01 09:11:58.70331 9.24 1 +2013-03-01 09:11:58.70331 9.24 1 +2013-03-01 09:11:58.70331 9.24 1 +2013-03-01 09:11:58.703311 7.38 1 +2013-03-01 09:11:58.703311 7.38 1 +2013-03-01 09:11:58.703311 7.38 1 +2013-03-01 09:11:58.703311 7.38 1 +2013-03-01 09:11:58.703311 7.38 1 +2013-03-01 09:11:58.703311 7.38 1 +2013-03-01 09:11:58.703312 18.2 1 +2013-03-01 09:11:58.703312 18.2 1 +2013-03-01 09:11:58.703312 18.2 1 +2013-03-01 09:11:58.703312 18.2 1 +2013-03-01 09:11:58.703312 18.2 1 +2013-03-01 09:11:58.703312 18.2 1 +2013-03-01 09:11:58.703313 9.35 1 +2013-03-01 09:11:58.703313 9.35 1 +2013-03-01 09:11:58.703313 9.35 1 +2013-03-01 09:11:58.703313 9.35 1 +2013-03-01 09:11:58.703313 9.35 1 +2013-03-01 09:11:58.703313 9.35 1 +2013-03-01 09:11:58.703313 9.35 1 +2013-03-01 09:11:58.703313 9.35 1 +2013-03-01 09:11:58.703314 39.12 1 +2013-03-01 09:11:58.703314 39.12 1 +2013-03-01 09:11:58.703314 39.12 1 +2013-03-01 09:11:58.703314 39.12 1 +2013-03-01 09:11:58.703315 4.93 1 +2013-03-01 09:11:58.703315 4.93 1 +2013-03-01 09:11:58.703315 4.93 1 +2013-03-01 09:11:58.703315 4.93 1 +2013-03-01 09:11:58.703315 4.93 1 +2013-03-01 09:11:58.703315 4.93 1 +2013-03-01 09:11:58.703315 4.93 1 +2013-03-01 09:11:58.703315 4.93 1 +2013-03-01 09:11:58.703316 16.86 1 +2013-03-01 09:11:58.703316 16.86 1 +2013-03-01 09:11:58.703316 16.86 1 +2013-03-01 09:11:58.703316 16.86 1 +2013-03-01 09:11:58.703316 16.86 1 +2013-03-01 09:11:58.703317 1.45 1 +2013-03-01 09:11:58.703317 1.45 1 +2013-03-01 09:11:58.703317 1.45 1 +2013-03-01 09:11:58.703317 1.45 1 +2013-03-01 09:11:58.703318 21.81 1 +2013-03-01 09:11:58.703318 21.81 1 +2013-03-01 09:11:58.703318 21.81 1 +2013-03-01 09:11:58.703318 21.81 1 +2013-03-01 09:11:58.703319 83.21 1 +2013-03-01 09:11:58.703319 83.21 1 +2013-03-01 09:11:58.703319 83.21 1 +2013-03-01 09:11:58.703319 83.21 1 +2013-03-01 09:11:58.703319 83.21 1 +2013-03-01 09:11:58.70332 77.09 1 +2013-03-01 09:11:58.70332 77.09 1 +2013-03-01 09:11:58.70332 77.09 1 +2013-03-01 09:11:58.70332 77.09 1 +2013-03-01 09:11:58.70332 77.09 1 +2013-03-01 09:11:58.70332 77.09 1 +2013-03-01 09:11:58.703321 3.91 1 +2013-03-01 09:11:58.703321 3.91 1 +2013-03-01 09:11:58.703321 3.91 1 +2013-03-01 09:11:58.703321 3.91 1 +2013-03-01 09:11:58.703322 2.48 1 +2013-03-01 09:11:58.703322 2.48 1 +2013-03-01 09:11:58.703322 2.48 1 +2013-03-01 09:11:58.703322 2.48 1 +2013-03-01 09:11:58.703322 2.48 1 +2013-03-01 09:11:58.703322 2.48 1 +2013-03-01 09:11:58.703323 36.22 1 +2013-03-01 09:11:58.703323 36.22 1 +2013-03-01 09:11:58.703323 36.22 1 +2013-03-01 09:11:58.703324 14.08 1 +2013-03-01 09:11:58.703324 14.08 1 +2013-03-01 09:11:58.703324 14.08 1 +2013-03-01 09:11:58.703324 14.08 1 +2013-03-01 09:11:58.703324 14.08 1 +2013-03-01 09:11:58.703324 14.08 1 +2013-03-01 09:11:58.703325 9.24 1 +2013-03-01 09:11:58.703325 9.24 1 +2013-03-01 09:11:58.703325 9.24 1 +2013-03-01 09:11:58.703325 9.24 1 +2013-03-01 09:11:58.703325 9.24 1 +2013-03-01 09:11:58.703325 9.24 1 diff --git a/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 3-0-58a982694ba2b1e34de82b1de54936a0 b/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 3-0-58a982694ba2b1e34de82b1de54936a0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 4-0-12cc78f3953c3e6b5411ddc729541bf0 b/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 4-0-12cc78f3953c3e6b5411ddc729541bf0 new file mode 100644 index 0000000000000..d02ca48857b5f --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 4-0-12cc78f3953c3e6b5411ddc729541bf0 @@ -0,0 +1,474 @@ +2013-03-01 09:11:58.703074 58.47 1 +2013-03-01 09:11:58.703074 58.47 1 +2013-03-01 09:11:58.703074 58.47 1 +2013-03-01 09:11:58.703074 58.47 1 +2013-03-01 09:11:58.703074 58.47 1 +2013-03-01 09:11:58.703074 58.47 1 +2013-03-01 09:11:58.703076 18.51 1 +2013-03-01 09:11:58.703076 18.51 1 +2013-03-01 09:11:58.703076 18.51 1 +2013-03-01 09:11:58.703076 18.51 1 +2013-03-01 09:11:58.703076 18.51 1 +2013-03-01 09:11:58.703076 18.51 1 +2013-03-01 09:11:58.703076 18.51 1 +2013-03-01 09:11:58.703077 66.68 1 +2013-03-01 09:11:58.703077 66.68 1 +2013-03-01 09:11:58.703077 66.68 1 +2013-03-01 09:11:58.703077 66.68 1 +2013-03-01 09:11:58.703087 25.19 1 +2013-03-01 09:11:58.703087 25.19 1 +2013-03-01 09:11:58.703087 25.19 1 +2013-03-01 09:11:58.703087 25.19 1 +2013-03-01 09:11:58.703087 25.19 1 +2013-03-01 09:11:58.703087 25.19 1 +2013-03-01 09:11:58.703087 25.19 1 +2013-03-01 09:11:58.703088 1.97 1 +2013-03-01 09:11:58.703088 1.97 1 +2013-03-01 09:11:58.703088 1.97 1 +2013-03-01 09:11:58.703088 1.97 1 +2013-03-01 09:11:58.703088 1.97 1 +2013-03-01 09:11:58.703088 1.97 1 +2013-03-01 09:11:58.703089 41.57 1 +2013-03-01 09:11:58.703089 41.57 1 +2013-03-01 09:11:58.703089 41.57 1 +2013-03-01 09:11:58.703089 41.57 1 +2013-03-01 09:11:58.703089 41.57 1 +2013-03-01 09:11:58.703089 41.57 1 +2013-03-01 09:11:58.703089 41.57 1 +2013-03-01 09:11:58.703091 68.85 1 +2013-03-01 09:11:58.703091 68.85 1 +2013-03-01 09:11:58.703091 68.85 1 +2013-03-01 09:11:58.703091 68.85 1 +2013-03-01 09:11:58.703091 68.85 1 +2013-03-01 09:11:58.703091 68.85 1 +2013-03-01 09:11:58.703091 68.85 1 +2013-03-01 09:11:58.703092 54.02 1 +2013-03-01 09:11:58.703092 54.02 1 +2013-03-01 09:11:58.703092 54.02 1 +2013-03-01 09:11:58.703096 87.84 1 +2013-03-01 09:11:58.703097 0.9 1 +2013-03-01 09:11:58.703097 0.9 1 +2013-03-01 09:11:58.703097 0.9 1 +2013-03-01 09:11:58.703098 21.29 1 +2013-03-01 09:11:58.703098 21.29 1 +2013-03-01 09:11:58.703098 21.29 1 +2013-03-01 09:11:58.703098 21.29 1 +2013-03-01 09:11:58.703098 21.29 1 +2013-03-01 09:11:58.703098 21.29 1 +2013-03-01 09:11:58.703098 21.29 1 +2013-03-01 09:11:58.703098 21.29 1 +2013-03-01 09:11:58.703098 21.29 1 +2013-03-01 09:11:58.703098 21.29 1 +2013-03-01 09:11:58.703104 75.85 1 +2013-03-01 09:11:58.703104 75.85 1 +2013-03-01 09:11:58.703104 75.85 1 +2013-03-01 09:11:58.703104 75.85 1 +2013-03-01 09:11:58.703104 75.85 1 +2013-03-01 09:11:58.70311 65.88 1 +2013-03-01 09:11:58.70311 65.88 1 +2013-03-01 09:11:58.70311 65.88 1 +2013-03-01 09:11:58.70311 65.88 1 +2013-03-01 09:11:58.703111 85.94 1 +2013-03-01 09:11:58.703111 85.94 1 +2013-03-01 09:11:58.703111 85.94 1 +2013-03-01 09:11:58.703111 85.94 1 +2013-03-01 09:11:58.703111 85.94 1 +2013-03-01 09:11:58.703111 85.94 1 +2013-03-01 09:11:58.703112 13.29 1 +2013-03-01 09:11:58.703112 13.29 1 +2013-03-01 09:11:58.703112 13.29 1 +2013-03-01 09:11:58.703112 13.29 1 +2013-03-01 09:11:58.703113 58.65 1 +2013-03-01 09:11:58.703113 58.65 1 +2013-03-01 09:11:58.703113 58.65 1 +2013-03-01 09:11:58.703113 58.65 1 +2013-03-01 09:11:58.703113 58.65 1 +2013-03-01 09:11:58.703113 58.65 1 +2013-03-01 09:11:58.703113 58.65 1 +2013-03-01 09:11:58.703113 58.65 1 +2013-03-01 09:11:58.703118 8.69 1 +2013-03-01 09:11:58.703118 8.69 1 +2013-03-01 09:11:58.70312 52.6 1 +2013-03-01 09:11:58.70312 52.6 1 +2013-03-01 09:11:58.70312 52.6 1 +2013-03-01 09:11:58.70312 52.6 1 +2013-03-01 09:11:58.703125 78.52 1 +2013-03-01 09:11:58.703125 78.52 1 +2013-03-01 09:11:58.703125 78.52 1 +2013-03-01 09:11:58.703125 78.52 1 +2013-03-01 09:11:58.703125 78.52 1 +2013-03-01 09:11:58.703125 78.52 1 +2013-03-01 09:11:58.703131 63.81 1 +2013-03-01 09:11:58.703131 63.81 1 +2013-03-01 09:11:58.703131 63.81 1 +2013-03-01 09:11:58.703131 63.81 1 +2013-03-01 09:11:58.703131 63.81 1 +2013-03-01 09:11:58.703131 63.81 1 +2013-03-01 09:11:58.703132 1.86 1 +2013-03-01 09:11:58.703132 1.86 1 +2013-03-01 09:11:58.703135 88.38 1 +2013-03-01 09:11:58.703135 88.38 1 +2013-03-01 09:11:58.703135 88.38 1 +2013-03-01 09:11:58.703135 88.38 1 +2013-03-01 09:11:58.703135 88.38 1 +2013-03-01 09:11:58.703135 88.38 1 +2013-03-01 09:11:58.703135 88.38 1 +2013-03-01 09:11:58.703135 88.38 1 +2013-03-01 09:11:58.703136 27.89 1 +2013-03-01 09:11:58.703136 27.89 1 +2013-03-01 09:11:58.703136 27.89 1 +2013-03-01 09:11:58.703136 27.89 1 +2013-03-01 09:11:58.703136 27.89 1 +2013-03-01 09:11:58.703138 86.7 1 +2013-03-01 09:11:58.703138 86.7 1 +2013-03-01 09:11:58.703138 86.7 1 +2013-03-01 09:11:58.703138 86.7 1 +2013-03-01 09:11:58.703138 86.7 1 +2013-03-01 09:11:58.703139 43.53 1 +2013-03-01 09:11:58.703139 43.53 1 +2013-03-01 09:11:58.703139 43.53 1 +2013-03-01 09:11:58.703139 43.53 1 +2013-03-01 09:11:58.703144 21.59 1 +2013-03-01 09:11:58.703144 21.59 1 +2013-03-01 09:11:58.703144 21.59 1 +2013-03-01 09:11:58.703144 21.59 1 +2013-03-01 09:11:58.703144 21.59 1 +2013-03-01 09:11:58.703144 21.59 1 +2013-03-01 09:11:58.703144 21.59 1 +2013-03-01 09:11:58.703154 16.93 1 +2013-03-01 09:11:58.703154 16.93 1 +2013-03-01 09:11:58.703154 16.93 1 +2013-03-01 09:11:58.703154 16.93 1 +2013-03-01 09:11:58.703154 16.93 1 +2013-03-01 09:11:58.703156 62.42 1 +2013-03-01 09:11:58.703156 62.42 1 +2013-03-01 09:11:58.703156 62.42 1 +2013-03-01 09:11:58.703156 62.42 1 +2013-03-01 09:11:58.703157 8.99 1 +2013-03-01 09:11:58.703157 8.99 1 +2013-03-01 09:11:58.703157 8.99 1 +2013-03-01 09:11:58.703157 8.99 1 +2013-03-01 09:11:58.703157 8.99 1 +2013-03-01 09:11:58.703157 8.99 1 +2013-03-01 09:11:58.703157 8.99 1 +2013-03-01 09:11:58.703157 8.99 1 +2013-03-01 09:11:58.703174 36.33 1 +2013-03-01 09:11:58.703174 36.33 1 +2013-03-01 09:11:58.703174 36.33 1 +2013-03-01 09:11:58.703174 36.33 1 +2013-03-01 09:11:58.703178 93.29 1 +2013-03-01 09:11:58.703178 93.29 1 +2013-03-01 09:11:58.703178 93.29 1 +2013-03-01 09:11:58.703178 93.29 1 +2013-03-01 09:11:58.703179 60.94 1 +2013-03-01 09:11:58.703179 60.94 1 +2013-03-01 09:11:58.703179 60.94 1 +2013-03-01 09:11:58.703179 60.94 1 +2013-03-01 09:11:58.703179 60.94 1 +2013-03-01 09:11:58.703179 60.94 1 +2013-03-01 09:11:58.703179 60.94 1 +2013-03-01 09:11:58.703181 26.6 1 +2013-03-01 09:11:58.703181 26.6 1 +2013-03-01 09:11:58.703181 26.6 1 +2013-03-01 09:11:58.703181 26.6 1 +2013-03-01 09:11:58.703181 26.6 1 +2013-03-01 09:11:58.703181 26.6 1 +2013-03-01 09:11:58.703181 26.6 1 +2013-03-01 09:11:58.703184 73.93 1 +2013-03-01 09:11:58.703184 73.93 1 +2013-03-01 09:11:58.703184 73.93 1 +2013-03-01 09:11:58.703184 73.93 1 +2013-03-01 09:11:58.703184 73.93 1 +2013-03-01 09:11:58.703185 8.91 1 +2013-03-01 09:11:58.703185 8.91 1 +2013-03-01 09:11:58.703185 8.91 1 +2013-03-01 09:11:58.703185 8.91 1 +2013-03-01 09:11:58.703185 8.91 1 +2013-03-01 09:11:58.703186 91.46 1 +2013-03-01 09:11:58.703186 91.46 1 +2013-03-01 09:11:58.703186 91.46 1 +2013-03-01 09:11:58.703186 91.46 1 +2013-03-01 09:11:58.703186 91.46 1 +2013-03-01 09:11:58.703186 91.46 1 +2013-03-01 09:11:58.703186 91.46 1 +2013-03-01 09:11:58.703186 91.46 1 +2013-03-01 09:11:58.703188 32.85 1 +2013-03-01 09:11:58.703188 32.85 1 +2013-03-01 09:11:58.703188 32.85 1 +2013-03-01 09:11:58.703188 32.85 1 +2013-03-01 09:11:58.703189 37.74 1 +2013-03-01 09:11:58.703189 37.74 1 +2013-03-01 09:11:58.703189 37.74 1 +2013-03-01 09:11:58.703189 37.74 1 +2013-03-01 09:11:58.703189 37.74 1 +2013-03-01 09:11:58.703189 37.74 1 +2013-03-01 09:11:58.703195 82.5 1 +2013-03-01 09:11:58.703195 82.5 1 +2013-03-01 09:11:58.703195 82.5 1 +2013-03-01 09:11:58.703195 82.5 1 +2013-03-01 09:11:58.703195 82.5 1 +2013-03-01 09:11:58.703195 82.5 1 +2013-03-01 09:11:58.703198 97.18 1 +2013-03-01 09:11:58.703198 97.18 1 +2013-03-01 09:11:58.703198 97.18 1 +2013-03-01 09:11:58.703198 97.18 1 +2013-03-01 09:11:58.703206 80.94 1 +2013-03-01 09:11:58.703206 80.94 1 +2013-03-01 09:11:58.703206 80.94 1 +2013-03-01 09:11:58.703206 80.94 1 +2013-03-01 09:11:58.703206 80.94 1 +2013-03-01 09:11:58.703206 80.94 1 +2013-03-01 09:11:58.703207 55.06 1 +2013-03-01 09:11:58.703207 55.06 1 +2013-03-01 09:11:58.703207 55.06 1 +2013-03-01 09:11:58.703207 55.06 1 +2013-03-01 09:11:58.703207 55.06 1 +2013-03-01 09:11:58.703209 25.92 1 +2013-03-01 09:11:58.703209 25.92 1 +2013-03-01 09:11:58.70321 37.12 1 +2013-03-01 09:11:58.70321 37.12 1 +2013-03-01 09:11:58.70321 37.12 1 +2013-03-01 09:11:58.70321 37.12 1 +2013-03-01 09:11:58.70321 37.12 1 +2013-03-01 09:11:58.70321 37.12 1 +2013-03-01 09:11:58.70321 37.12 1 +2013-03-01 09:11:58.70321 37.12 1 +2013-03-01 09:11:58.703213 48.8 1 +2013-03-01 09:11:58.703213 48.8 1 +2013-03-01 09:11:58.703213 48.8 1 +2013-03-01 09:11:58.703213 48.8 1 +2013-03-01 09:11:58.703219 32.73 1 +2013-03-01 09:11:58.703219 32.73 1 +2013-03-01 09:11:58.703219 32.73 1 +2013-03-01 09:11:58.703219 32.73 1 +2013-03-01 09:11:58.703219 32.73 1 +2013-03-01 09:11:58.703219 32.73 1 +2013-03-01 09:11:58.70322 7.37 1 +2013-03-01 09:11:58.70322 7.37 1 +2013-03-01 09:11:58.70322 7.37 1 +2013-03-01 09:11:58.703221 26.64 1 +2013-03-01 09:11:58.703221 26.64 1 +2013-03-01 09:11:58.703221 26.64 1 +2013-03-01 09:11:58.703221 26.64 1 +2013-03-01 09:11:58.703221 26.64 1 +2013-03-01 09:11:58.703221 26.64 1 +2013-03-01 09:11:58.703221 26.64 1 +2013-03-01 09:11:58.703223 57.1 1 +2013-03-01 09:11:58.703223 57.1 1 +2013-03-01 09:11:58.703223 57.1 1 +2013-03-01 09:11:58.703223 57.1 1 +2013-03-01 09:11:58.703223 57.1 1 +2013-03-01 09:11:58.703223 57.1 1 +2013-03-01 09:11:58.703223 57.1 1 +2013-03-01 09:11:58.703224 42.93 1 +2013-03-01 09:11:58.703224 42.93 1 +2013-03-01 09:11:58.703224 42.93 1 +2013-03-01 09:11:58.703224 42.93 1 +2013-03-01 09:11:58.703226 68.3 1 +2013-03-01 09:11:58.703226 68.3 1 +2013-03-01 09:11:58.703226 68.3 1 +2013-03-01 09:11:58.703226 68.3 1 +2013-03-01 09:11:58.703226 68.3 1 +2013-03-01 09:11:58.703226 68.3 1 +2013-03-01 09:11:58.703231 18.7 1 +2013-03-01 09:11:58.703231 18.7 1 +2013-03-01 09:11:58.703231 18.7 1 +2013-03-01 09:11:58.703231 18.7 1 +2013-03-01 09:11:58.703231 18.7 1 +2013-03-01 09:11:58.703231 18.7 1 +2013-03-01 09:11:58.703231 18.7 1 +2013-03-01 09:11:58.703233 40.81 1 +2013-03-01 09:11:58.703233 40.81 1 +2013-03-01 09:11:58.703233 40.81 1 +2013-03-01 09:11:58.703233 40.81 1 +2013-03-01 09:11:58.703233 40.81 1 +2013-03-01 09:11:58.703233 40.81 1 +2013-03-01 09:11:58.703242 31.23 1 +2013-03-01 09:11:58.703242 31.23 1 +2013-03-01 09:11:58.703242 31.23 1 +2013-03-01 09:11:58.703242 31.23 1 +2013-03-01 09:11:58.703244 25.67 1 +2013-03-01 09:11:58.703244 25.67 1 +2013-03-01 09:11:58.703244 25.67 1 +2013-03-01 09:11:58.703244 25.67 1 +2013-03-01 09:11:58.703244 25.67 1 +2013-03-01 09:11:58.703244 25.67 1 +2013-03-01 09:11:58.703244 25.67 1 +2013-03-01 09:11:58.703244 25.67 1 +2013-03-01 09:11:58.703245 32.3 1 +2013-03-01 09:11:58.703245 32.3 1 +2013-03-01 09:11:58.703245 32.3 1 +2013-03-01 09:11:58.703245 32.3 1 +2013-03-01 09:11:58.703245 32.3 1 +2013-03-01 09:11:58.703245 32.3 1 +2013-03-01 09:11:58.703245 32.3 1 +2013-03-01 09:11:58.703245 32.3 1 +2013-03-01 09:11:58.703245 32.3 1 +2013-03-01 09:11:58.703246 72.87 1 +2013-03-01 09:11:58.703246 72.87 1 +2013-03-01 09:11:58.703248 81.28 1 +2013-03-01 09:11:58.703248 81.28 1 +2013-03-01 09:11:58.703248 81.28 1 +2013-03-01 09:11:58.703249 93.3 1 +2013-03-01 09:11:58.703249 93.3 1 +2013-03-01 09:11:58.703249 93.3 1 +2013-03-01 09:11:58.703249 93.3 1 +2013-03-01 09:11:58.70325 93.79 1 +2013-03-01 09:11:58.70325 93.79 1 +2013-03-01 09:11:58.70325 93.79 1 +2013-03-01 09:11:58.70325 93.79 1 +2013-03-01 09:11:58.70325 93.79 1 +2013-03-01 09:11:58.70325 93.79 1 +2013-03-01 09:11:58.70325 93.79 1 +2013-03-01 09:11:58.703254 0.32 1 +2013-03-01 09:11:58.703254 0.32 1 +2013-03-01 09:11:58.703254 0.32 1 +2013-03-01 09:11:58.703254 0.32 1 +2013-03-01 09:11:58.703256 43.8 1 +2013-03-01 09:11:58.703256 43.8 1 +2013-03-01 09:11:58.703256 43.8 1 +2013-03-01 09:11:58.703256 43.8 1 +2013-03-01 09:11:58.703256 43.8 1 +2013-03-01 09:11:58.703256 43.8 1 +2013-03-01 09:11:58.703258 21.21 1 +2013-03-01 09:11:58.703258 21.21 1 +2013-03-01 09:11:58.703258 21.21 1 +2013-03-01 09:11:58.703259 52.28 1 +2013-03-01 09:11:58.703259 52.28 1 +2013-03-01 09:11:58.703259 52.28 1 +2013-03-01 09:11:58.703259 52.28 1 +2013-03-01 09:11:58.703259 52.28 1 +2013-03-01 09:11:58.703259 52.28 1 +2013-03-01 09:11:58.703262 78.56 1 +2013-03-01 09:11:58.703262 78.56 1 +2013-03-01 09:11:58.703262 78.56 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703264 52.49 1 +2013-03-01 09:11:58.703264 52.49 1 +2013-03-01 09:11:58.703264 52.49 1 +2013-03-01 09:11:58.703264 52.49 1 +2013-03-01 09:11:58.703265 11.46 1 +2013-03-01 09:11:58.703265 11.46 1 +2013-03-01 09:11:58.703265 11.46 1 +2013-03-01 09:11:58.703265 11.46 1 +2013-03-01 09:11:58.703266 83.67 1 +2013-03-01 09:11:58.703266 83.67 1 +2013-03-01 09:11:58.703266 83.67 1 +2013-03-01 09:11:58.703266 83.67 1 +2013-03-01 09:11:58.703266 83.67 1 +2013-03-01 09:11:58.703269 61.06 1 +2013-03-01 09:11:58.703269 61.06 1 +2013-03-01 09:11:58.703269 61.06 1 +2013-03-01 09:11:58.703269 61.06 1 +2013-03-01 09:11:58.703269 61.06 1 +2013-03-01 09:11:58.703272 7.49 1 +2013-03-01 09:11:58.703272 7.49 1 +2013-03-01 09:11:58.703272 7.49 1 +2013-03-01 09:11:58.703272 7.49 1 +2013-03-01 09:11:58.703273 30.49 1 +2013-03-01 09:11:58.703273 30.49 1 +2013-03-01 09:11:58.703273 30.49 1 +2013-03-01 09:11:58.703275 90.69 1 +2013-03-01 09:11:58.703275 90.69 1 +2013-03-01 09:11:58.703275 90.69 1 +2013-03-01 09:11:58.703275 90.69 1 +2013-03-01 09:11:58.703275 90.69 1 +2013-03-01 09:11:58.703275 90.69 1 +2013-03-01 09:11:58.703276 88.46 1 +2013-03-01 09:11:58.703276 88.46 1 +2013-03-01 09:11:58.703276 88.46 1 +2013-03-01 09:11:58.703276 88.46 1 +2013-03-01 09:11:58.703278 69.42 1 +2013-03-01 09:11:58.703278 69.42 1 +2013-03-01 09:11:58.703278 69.42 1 +2013-03-01 09:11:58.70328 45.81 1 +2013-03-01 09:11:58.70328 45.81 1 +2013-03-01 09:11:58.70328 45.81 1 +2013-03-01 09:11:58.70328 45.81 1 +2013-03-01 09:11:58.70328 45.81 1 +2013-03-01 09:11:58.70328 45.81 1 +2013-03-01 09:11:58.703281 62.11 1 +2013-03-01 09:11:58.703281 62.11 1 +2013-03-01 09:11:58.703281 62.11 1 +2013-03-01 09:11:58.703281 62.11 1 +2013-03-01 09:11:58.703281 62.11 1 +2013-03-01 09:11:58.703283 17.62 1 +2013-03-01 09:11:58.703283 17.62 1 +2013-03-01 09:11:58.703283 17.62 1 +2013-03-01 09:11:58.703283 17.62 1 +2013-03-01 09:11:58.703283 17.62 1 +2013-03-01 09:11:58.703283 17.62 1 +2013-03-01 09:11:58.703283 17.62 1 +2013-03-01 09:11:58.703283 17.62 1 +2013-03-01 09:11:58.703289 0.56 1 +2013-03-01 09:11:58.703289 0.56 1 +2013-03-01 09:11:58.703289 0.56 1 +2013-03-01 09:11:58.703289 0.56 1 +2013-03-01 09:11:58.703289 0.56 1 +2013-03-01 09:11:58.703289 0.56 1 +2013-03-01 09:11:58.703289 0.56 1 +2013-03-01 09:11:58.703293 42.86 1 +2013-03-01 09:11:58.703293 42.86 1 +2013-03-01 09:11:58.703293 42.86 1 +2013-03-01 09:11:58.703294 29.74 1 +2013-03-01 09:11:58.703294 29.74 1 +2013-03-01 09:11:58.703294 29.74 1 +2013-03-01 09:11:58.703294 29.74 1 +2013-03-01 09:11:58.703294 29.74 1 +2013-03-01 09:11:58.703294 29.74 1 +2013-03-01 09:11:58.703296 43.66 1 +2013-03-01 09:11:58.703296 43.66 1 +2013-03-01 09:11:58.703296 43.66 1 +2013-03-01 09:11:58.703296 43.66 1 +2013-03-01 09:11:58.703296 43.66 1 +2013-03-01 09:11:58.703296 43.66 1 +2013-03-01 09:11:58.703296 43.66 1 +2013-03-01 09:11:58.703296 43.66 1 +2013-03-01 09:11:58.703299 23.19 1 +2013-03-01 09:11:58.703299 23.19 1 +2013-03-01 09:11:58.703299 23.19 1 +2013-03-01 09:11:58.703299 23.19 1 +2013-03-01 09:11:58.703299 23.19 1 +2013-03-01 09:11:58.703309 89.4 1 +2013-03-01 09:11:58.703309 89.4 1 +2013-03-01 09:11:58.703309 89.4 1 +2013-03-01 09:11:58.703309 89.4 1 +2013-03-01 09:11:58.70331 69.26 1 +2013-03-01 09:11:58.70331 69.26 1 +2013-03-01 09:11:58.70331 69.26 1 +2013-03-01 09:11:58.703313 20.69 1 +2013-03-01 09:11:58.703313 20.69 1 +2013-03-01 09:11:58.703313 20.69 1 +2013-03-01 09:11:58.703313 20.69 1 +2013-03-01 09:11:58.703315 53.04 1 +2013-03-01 09:11:58.703315 53.04 1 +2013-03-01 09:11:58.703315 53.04 1 +2013-03-01 09:11:58.703315 53.04 1 +2013-03-01 09:11:58.703318 85.62 1 +2013-03-01 09:11:58.703318 85.62 1 +2013-03-01 09:11:58.703318 85.62 1 +2013-03-01 09:11:58.703323 65.61 1 +2013-03-01 09:11:58.703323 65.61 1 +2013-03-01 09:11:58.703323 65.61 1 +2013-03-01 09:11:58.703323 65.61 1 +2013-03-01 09:11:58.703323 65.61 1 +2013-03-01 09:11:58.703323 65.61 1 +2013-03-01 09:11:58.703323 65.61 1 +2013-03-01 09:11:58.703323 65.61 1 +2013-03-01 09:11:58.703324 98.36 1 +2013-03-01 09:11:58.703324 98.36 1 +2013-03-01 09:11:58.703324 98.36 1 +2013-03-01 09:11:58.703324 98.36 1 +2013-03-01 09:11:58.703325 65.81 1 +2013-03-01 09:11:58.703325 65.81 1 +2013-03-01 09:11:58.703325 65.81 1 +2013-03-01 09:11:58.703325 65.81 1 diff --git a/sql/hive/src/test/resources/golden/windowing_udaf.q (deterministic)-0-6642a21d87e0401ba1a668ea8b244f0c b/sql/hive/src/test/resources/golden/windowing_udaf.q (deterministic)-0-6642a21d87e0401ba1a668ea8b244f0c new file mode 100644 index 0000000000000..119dd71df1426 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_udaf.q (deterministic)-0-6642a21d87e0401ba1a668ea8b244f0c @@ -0,0 +1,1049 @@ + 65560 + 65560 + 65560 +alice allen 65662 +alice allen 65662 +alice allen 65662 +alice brown 65696 +alice carson 65559 +alice davidson 65547 +alice falkner 65669 +alice garcia 65613 +alice hernandez 65737 +alice hernandez 65737 +alice johnson 65739 +alice king 65660 +alice king 65660 +alice king 65660 +alice laertes 65669 +alice laertes 65669 +alice miller 65590 +alice nixon 65586 +alice nixon 65586 +alice nixon 65586 +alice ovid 65737 +alice polk 65548 +alice quirinius 65636 +alice quirinius 65636 +alice robinson 65606 +alice robinson 65606 +alice steinbeck 65578 +alice steinbeck 65578 +alice steinbeck 65578 +alice underhill 65750 +alice van buren 65562 +alice xylophone 65578 +alice xylophone 65578 +alice xylophone 65578 +alice zipper 65553 +alice zipper 65553 +alice zipper 65553 +bob brown 65584 +bob brown 65584 +bob brown 65584 +bob carson 65713 +bob davidson 65664 +bob davidson 65664 +bob davidson 65664 +bob ellison 65591 +bob ellison 65591 +bob ellison 65591 +bob ellison 65591 +bob falkner 65789 +bob garcia 65585 +bob garcia 65585 +bob garcia 65585 +bob garcia 65585 +bob garcia 65585 +bob hernandez 65557 +bob ichabod 65549 +bob king 65715 +bob king 65715 +bob king 65715 +bob laertes 65602 +bob laertes 65602 +bob miller 65608 +bob ovid 65564 +bob ovid 65564 +bob ovid 65564 +bob ovid 65564 +bob polk 65594 +bob quirinius 65700 +bob steinbeck 65637 +bob van buren 65778 +bob white 65543 +bob white 65543 +bob xylophone 65574 +bob xylophone 65574 +bob young 65556 +bob zipper 65559 +bob zipper 65559 +bob zipper 65559 +calvin allen 65669 +calvin brown 65537 +calvin brown 65537 +calvin brown 65537 +calvin carson 65637 +calvin davidson 65541 +calvin davidson 65541 +calvin ellison 65667 +calvin falkner 65573 +calvin falkner 65573 +calvin falkner 65573 +calvin falkner 65573 +calvin falkner 65573 +calvin falkner 65573 +calvin garcia 65664 +calvin hernandez 65578 +calvin johnson 65731 +calvin laertes 65570 +calvin laertes 65570 +calvin nixon 65654 +calvin nixon 65654 +calvin nixon 65654 +calvin ovid 65554 +calvin ovid 65554 +calvin ovid 65554 +calvin ovid 65554 +calvin polk 65731 +calvin quirinius 65741 +calvin quirinius 65741 +calvin robinson 65581 +calvin steinbeck 65680 +calvin steinbeck 65680 +calvin steinbeck 65680 +calvin thompson 65560 +calvin thompson 65560 +calvin underhill 65732 +calvin van buren 65552 +calvin van buren 65552 +calvin white 65553 +calvin white 65553 +calvin xylophone 65575 +calvin xylophone 65575 +calvin xylophone 65575 +calvin young 65574 +calvin young 65574 +calvin zipper 65669 +calvin zipper 65669 +david allen 65588 +david allen 65588 +david brown 65637 +david brown 65637 +david davidson 65559 +david davidson 65559 +david davidson 65559 +david davidson 65559 +david ellison 65634 +david ellison 65634 +david ellison 65634 +david hernandez 65763 +david ichabod 65699 +david ichabod 65699 +david laertes 65762 +david nixon 65536 +david ovid 65623 +david ovid 65623 +david quirinius 65697 +david quirinius 65697 +david quirinius 65697 +david robinson 65762 +david robinson 65762 +david thompson 65550 +david underhill 65602 +david underhill 65602 +david underhill 65602 +david van buren 65625 +david van buren 65625 +david white 65678 +david xylophone 65537 +david xylophone 65537 +david xylophone 65537 +david young 65551 +david young 65551 +ethan allen 65747 +ethan brown 65539 +ethan brown 65539 +ethan brown 65539 +ethan brown 65539 +ethan brown 65539 +ethan brown 65539 +ethan carson 65742 +ethan ellison 65714 +ethan ellison 65714 +ethan falkner 65577 +ethan falkner 65577 +ethan garcia 65736 +ethan hernandez 65618 +ethan johnson 65536 +ethan king 65614 +ethan laertes 65562 +ethan laertes 65562 +ethan laertes 65562 +ethan laertes 65562 +ethan laertes 65562 +ethan laertes 65562 +ethan laertes 65562 +ethan miller 65712 +ethan nixon 65766 +ethan ovid 65697 +ethan polk 65589 +ethan polk 65589 +ethan polk 65589 +ethan polk 65589 +ethan quirinius 65542 +ethan quirinius 65542 +ethan quirinius 65542 +ethan robinson 65547 +ethan robinson 65547 +ethan underhill 65570 +ethan van buren 65572 +ethan white 65677 +ethan white 65677 +ethan xylophone 65595 +ethan zipper 65593 +ethan zipper 65593 +fred davidson 65552 +fred davidson 65552 +fred davidson 65552 +fred ellison 65548 +fred ellison 65548 +fred ellison 65548 +fred falkner 65637 +fred falkner 65637 +fred falkner 65637 +fred hernandez 65541 +fred ichabod 65572 +fred ichabod 65572 +fred johnson 65758 +fred king 65694 +fred king 65694 +fred laertes 65769 +fred miller 65536 +fred nixon 65560 +fred nixon 65560 +fred nixon 65560 +fred nixon 65560 +fred polk 65603 +fred polk 65603 +fred polk 65603 +fred polk 65603 +fred quirinius 65697 +fred quirinius 65697 +fred robinson 65623 +fred steinbeck 65544 +fred steinbeck 65544 +fred steinbeck 65544 +fred underhill 65629 +fred van buren 65537 +fred van buren 65537 +fred van buren 65537 +fred van buren 65537 +fred white 65589 +fred young 65594 +fred young 65594 +fred zipper 65553 +gabriella allen 65646 +gabriella allen 65646 +gabriella brown 65704 +gabriella brown 65704 +gabriella carson 65586 +gabriella davidson 65565 +gabriella ellison 65706 +gabriella ellison 65706 +gabriella falkner 65623 +gabriella falkner 65623 +gabriella falkner 65623 +gabriella garcia 65571 +gabriella hernandez 65587 +gabriella hernandez 65587 +gabriella ichabod 65559 +gabriella ichabod 65559 +gabriella ichabod 65559 +gabriella ichabod 65559 +gabriella ichabod 65559 +gabriella king 65651 +gabriella king 65651 +gabriella laertes 65781 +gabriella miller 65646 +gabriella ovid 65556 +gabriella ovid 65556 +gabriella polk 65701 +gabriella polk 65701 +gabriella steinbeck 65582 +gabriella steinbeck 65582 +gabriella thompson 65682 +gabriella thompson 65682 +gabriella thompson 65682 +gabriella van buren 65581 +gabriella van buren 65581 +gabriella white 65638 +gabriella young 65699 +gabriella young 65699 +gabriella zipper 65540 +gabriella zipper 65540 +holly allen 65596 +holly brown 65599 +holly brown 65599 +holly falkner 65720 +holly hernandez 65602 +holly hernandez 65602 +holly hernandez 65602 +holly hernandez 65602 +holly ichabod 65711 +holly ichabod 65711 +holly ichabod 65711 +holly johnson 65655 +holly johnson 65655 +holly johnson 65655 +holly king 65549 +holly king 65549 +holly laertes 65664 +holly miller 65653 +holly nixon 65539 +holly nixon 65539 +holly polk 65743 +holly polk 65743 +holly robinson 65564 +holly thompson 65538 +holly thompson 65538 +holly thompson 65538 +holly underhill 65634 +holly underhill 65634 +holly underhill 65634 +holly underhill 65634 +holly van buren 65727 +holly white 65536 +holly white 65536 +holly xylophone 65544 +holly young 65606 +holly young 65606 +holly zipper 65607 +holly zipper 65607 +irene allen 65556 +irene brown 65633 +irene brown 65633 +irene brown 65633 +irene carson 65590 +irene ellison 65659 +irene ellison 65659 +irene falkner 65620 +irene falkner 65620 +irene garcia 65660 +irene garcia 65660 +irene garcia 65660 +irene ichabod 65645 +irene ichabod 65645 +irene johnson 65583 +irene laertes 65664 +irene laertes 65664 +irene laertes 65664 +irene miller 65730 +irene nixon 65631 +irene nixon 65631 +irene nixon 65631 +irene ovid 65691 +irene ovid 65691 +irene ovid 65691 +irene polk 65551 +irene polk 65551 +irene polk 65551 +irene polk 65551 +irene polk 65551 +irene quirinius 65724 +irene quirinius 65724 +irene quirinius 65724 +irene robinson 65554 +irene steinbeck 65683 +irene thompson 65688 +irene underhill 65591 +irene underhill 65591 +irene van buren 65579 +irene van buren 65579 +irene xylophone 65775 +jessica brown 65588 +jessica carson 65553 +jessica carson 65553 +jessica carson 65553 +jessica davidson 65549 +jessica davidson 65549 +jessica davidson 65549 +jessica davidson 65549 +jessica ellison 65567 +jessica ellison 65567 +jessica falkner 65584 +jessica garcia 65676 +jessica garcia 65676 +jessica ichabod 65704 +jessica johnson 65607 +jessica johnson 65607 +jessica miller 65733 +jessica nixon 65590 +jessica nixon 65590 +jessica ovid 65582 +jessica ovid 65582 +jessica polk 65637 +jessica quirinius 65562 +jessica quirinius 65562 +jessica quirinius 65562 +jessica quirinius 65562 +jessica robinson 65576 +jessica thompson 65581 +jessica thompson 65581 +jessica underhill 65656 +jessica underhill 65656 +jessica underhill 65656 +jessica van buren 65615 +jessica white 65544 +jessica white 65544 +jessica white 65544 +jessica white 65544 +jessica white 65544 +jessica xylophone 65562 +jessica young 65623 +jessica young 65623 +jessica zipper 65600 +jessica zipper 65600 +jessica zipper 65600 +katie allen 65542 +katie brown 65590 +katie davidson 65619 +katie ellison 65675 +katie ellison 65675 +katie falkner 65728 +katie garcia 65625 +katie garcia 65625 +katie hernandez 65550 +katie ichabod 65658 +katie ichabod 65658 +katie ichabod 65658 +katie king 65629 +katie king 65629 +katie king 65629 +katie miller 65541 +katie miller 65541 +katie nixon 65669 +katie ovid 65681 +katie polk 65746 +katie polk 65746 +katie robinson 65697 +katie van buren 65643 +katie van buren 65643 +katie white 65620 +katie white 65620 +katie xylophone 65585 +katie young 65644 +katie young 65644 +katie young 65644 +katie zipper 65568 +katie zipper 65568 +luke allen 65547 +luke allen 65547 +luke allen 65547 +luke allen 65547 +luke allen 65547 +luke brown 65719 +luke davidson 65656 +luke davidson 65656 +luke ellison 65582 +luke ellison 65582 +luke ellison 65582 +luke falkner 65589 +luke falkner 65589 +luke garcia 65687 +luke garcia 65687 +luke ichabod 65629 +luke ichabod 65629 +luke johnson 65545 +luke johnson 65545 +luke johnson 65545 +luke laertes 65608 +luke laertes 65608 +luke laertes 65608 +luke laertes 65608 +luke laertes 65608 +luke miller 65752 +luke ovid 65569 +luke ovid 65569 +luke polk 65645 +luke polk 65645 +luke quirinius 65655 +luke robinson 65634 +luke robinson 65634 +luke thompson 65626 +luke underhill 65553 +luke underhill 65553 +luke underhill 65553 +luke van buren 65678 +luke white 65693 +luke xylophone 65597 +luke zipper 65641 +mike allen 65706 +mike brown 65654 +mike carson 65698 +mike carson 65698 +mike carson 65698 +mike davidson 65658 +mike davidson 65658 +mike ellison 65598 +mike ellison 65598 +mike ellison 65598 +mike ellison 65598 +mike ellison 65598 +mike falkner 65609 +mike garcia 65571 +mike garcia 65571 +mike garcia 65571 +mike hernandez 65548 +mike hernandez 65548 +mike ichabod 65621 +mike king 65563 +mike king 65563 +mike king 65563 +mike king 65563 +mike king 65563 +mike king 65563 +mike miller 65549 +mike nixon 65619 +mike nixon 65619 +mike polk 65619 +mike polk 65619 +mike polk 65619 +mike quirinius 65717 +mike steinbeck 65550 +mike steinbeck 65550 +mike steinbeck 65550 +mike steinbeck 65550 +mike van buren 65620 +mike van buren 65620 +mike white 65648 +mike white 65648 +mike white 65648 +mike white 65648 +mike young 65545 +mike young 65545 +mike young 65545 +mike zipper 65552 +mike zipper 65552 +mike zipper 65552 +nick allen 65641 +nick allen 65641 +nick brown 65724 +nick davidson 65601 +nick ellison 65691 +nick ellison 65691 +nick falkner 65583 +nick falkner 65583 +nick garcia 65695 +nick garcia 65695 +nick garcia 65695 +nick ichabod 65572 +nick ichabod 65572 +nick ichabod 65572 +nick johnson 65585 +nick johnson 65585 +nick laertes 65624 +nick miller 65757 +nick nixon 65650 +nick ovid 65719 +nick polk 65716 +nick quirinius 65588 +nick quirinius 65588 +nick robinson 65547 +nick robinson 65547 +nick steinbeck 65689 +nick thompson 65610 +nick underhill 65619 +nick van buren 65603 +nick xylophone 65644 +nick young 65654 +nick young 65654 +nick zipper 65757 +nick zipper 65757 +oscar allen 65644 +oscar brown 65614 +oscar carson 65537 +oscar carson 65537 +oscar carson 65537 +oscar carson 65537 +oscar carson 65537 +oscar davidson 65556 +oscar ellison 65630 +oscar ellison 65630 +oscar falkner 65692 +oscar garcia 65751 +oscar hernandez 65683 +oscar hernandez 65683 +oscar ichabod 65536 +oscar ichabod 65536 +oscar ichabod 65536 +oscar ichabod 65536 +oscar johnson 65645 +oscar johnson 65645 +oscar king 65541 +oscar king 65541 +oscar king 65541 +oscar laertes 65625 +oscar laertes 65625 +oscar laertes 65625 +oscar laertes 65625 +oscar nixon 65596 +oscar ovid 65536 +oscar ovid 65536 +oscar ovid 65536 +oscar polk 65541 +oscar polk 65541 +oscar quirinius 65541 +oscar quirinius 65541 +oscar quirinius 65541 +oscar quirinius 65541 +oscar robinson 65537 +oscar robinson 65537 +oscar robinson 65537 +oscar robinson 65537 +oscar steinbeck 65709 +oscar thompson 65542 +oscar thompson 65542 +oscar thompson 65542 +oscar thompson 65542 +oscar underhill 65626 +oscar van buren 65581 +oscar van buren 65581 +oscar van buren 65581 +oscar white 65552 +oscar white 65552 +oscar white 65552 +oscar white 65552 +oscar xylophone 65773 +oscar xylophone 65773 +oscar xylophone 65773 +oscar zipper 65568 +oscar zipper 65568 +oscar zipper 65568 +priscilla brown 65670 +priscilla brown 65670 +priscilla brown 65670 +priscilla carson 65658 +priscilla carson 65658 +priscilla carson 65658 +priscilla ichabod 65627 +priscilla ichabod 65627 +priscilla johnson 65543 +priscilla johnson 65543 +priscilla johnson 65543 +priscilla johnson 65543 +priscilla johnson 65543 +priscilla king 65646 +priscilla nixon 65564 +priscilla nixon 65564 +priscilla ovid 65541 +priscilla ovid 65541 +priscilla polk 65747 +priscilla quirinius 65672 +priscilla thompson 65654 +priscilla underhill 65715 +priscilla underhill 65715 +priscilla van buren 65607 +priscilla van buren 65607 +priscilla van buren 65607 +priscilla white 65652 +priscilla xylophone 65538 +priscilla xylophone 65538 +priscilla xylophone 65538 +priscilla young 65585 +priscilla young 65585 +priscilla zipper 65622 +priscilla zipper 65622 +quinn allen 65657 +quinn allen 65657 +quinn brown 65691 +quinn brown 65691 +quinn brown 65691 +quinn davidson 65549 +quinn davidson 65549 +quinn davidson 65549 +quinn davidson 65549 +quinn ellison 65705 +quinn ellison 65705 +quinn garcia 65568 +quinn garcia 65568 +quinn garcia 65568 +quinn garcia 65568 +quinn ichabod 65564 +quinn king 65558 +quinn king 65558 +quinn laertes 65542 +quinn laertes 65542 +quinn laertes 65542 +quinn nixon 65659 +quinn ovid 65699 +quinn quirinius 65747 +quinn robinson 65627 +quinn steinbeck 65578 +quinn steinbeck 65578 +quinn thompson 65643 +quinn thompson 65643 +quinn underhill 65549 +quinn underhill 65549 +quinn underhill 65549 +quinn van buren 65725 +quinn young 65647 +quinn zipper 65579 +quinn zipper 65579 +rachel allen 65661 +rachel allen 65661 +rachel brown 65586 +rachel brown 65586 +rachel brown 65586 +rachel brown 65586 +rachel brown 65586 +rachel carson 65677 +rachel carson 65677 +rachel davidson 65755 +rachel ellison 65761 +rachel falkner 65616 +rachel falkner 65616 +rachel falkner 65616 +rachel falkner 65616 +rachel johnson 65658 +rachel king 65604 +rachel king 65604 +rachel laertes 65562 +rachel laertes 65562 +rachel ovid 65721 +rachel ovid 65721 +rachel polk 65686 +rachel quirinius 65787 +rachel robinson 65544 +rachel robinson 65544 +rachel robinson 65544 +rachel thompson 65648 +rachel thompson 65648 +rachel thompson 65648 +rachel underhill 65667 +rachel white 65615 +rachel white 65615 +rachel young 65727 +rachel zipper 65757 +rachel zipper 65757 +sarah carson 65679 +sarah carson 65679 +sarah carson 65679 +sarah ellison 65611 +sarah falkner 65606 +sarah falkner 65606 +sarah garcia 65563 +sarah garcia 65563 +sarah garcia 65563 +sarah ichabod 65667 +sarah ichabod 65667 +sarah johnson 65659 +sarah johnson 65659 +sarah johnson 65659 +sarah johnson 65659 +sarah king 65650 +sarah king 65650 +sarah miller 65557 +sarah ovid 65550 +sarah robinson 65677 +sarah robinson 65677 +sarah steinbeck 65721 +sarah white 65622 +sarah white 65622 +sarah xylophone 65678 +sarah young 65595 +sarah zipper 65550 +tom brown 65593 +tom brown 65593 +tom carson 65539 +tom carson 65539 +tom carson 65539 +tom davidson 65780 +tom ellison 65578 +tom ellison 65578 +tom ellison 65578 +tom falkner 65574 +tom falkner 65574 +tom hernandez 65575 +tom hernandez 65575 +tom ichabod 65588 +tom johnson 65536 +tom johnson 65536 +tom king 65576 +tom laertes 65617 +tom laertes 65617 +tom miller 65594 +tom miller 65594 +tom miller 65594 +tom nixon 65672 +tom ovid 65628 +tom polk 65652 +tom polk 65652 +tom quirinius 65563 +tom quirinius 65563 +tom robinson 65626 +tom robinson 65626 +tom robinson 65626 +tom robinson 65626 +tom steinbeck 65666 +tom van buren 65621 +tom van buren 65621 +tom van buren 65621 +tom white 65548 +tom young 65544 +tom young 65544 +tom zipper 65789 +ulysses brown 65735 +ulysses carson 65602 +ulysses carson 65602 +ulysses carson 65602 +ulysses carson 65602 +ulysses davidson 65750 +ulysses ellison 65575 +ulysses garcia 65666 +ulysses hernandez 65651 +ulysses hernandez 65651 +ulysses hernandez 65651 +ulysses ichabod 65551 +ulysses ichabod 65551 +ulysses johnson 65776 +ulysses king 65649 +ulysses laertes 65691 +ulysses laertes 65691 +ulysses laertes 65691 +ulysses miller 65610 +ulysses miller 65610 +ulysses nixon 65603 +ulysses ovid 65656 +ulysses polk 65563 +ulysses polk 65563 +ulysses polk 65563 +ulysses polk 65563 +ulysses quirinius 65786 +ulysses robinson 65744 +ulysses steinbeck 65611 +ulysses steinbeck 65611 +ulysses thompson 65788 +ulysses underhill 65570 +ulysses underhill 65570 +ulysses underhill 65570 +ulysses underhill 65570 +ulysses underhill 65570 +ulysses underhill 65570 +ulysses underhill 65570 +ulysses van buren 65684 +ulysses white 65654 +ulysses white 65654 +ulysses xylophone 65623 +ulysses xylophone 65623 +ulysses xylophone 65623 +ulysses young 65675 +ulysses young 65675 +ulysses young 65675 +victor allen 65684 +victor allen 65684 +victor brown 65550 +victor brown 65550 +victor brown 65550 +victor brown 65550 +victor davidson 65579 +victor davidson 65579 +victor davidson 65579 +victor ellison 65641 +victor ellison 65641 +victor hernandez 65571 +victor hernandez 65571 +victor hernandez 65571 +victor hernandez 65571 +victor hernandez 65571 +victor johnson 65606 +victor johnson 65606 +victor johnson 65606 +victor king 65721 +victor king 65721 +victor laertes 65638 +victor laertes 65638 +victor miller 65570 +victor nixon 65709 +victor nixon 65709 +victor ovid 65649 +victor polk 65625 +victor quirinius 65620 +victor quirinius 65620 +victor robinson 65596 +victor robinson 65596 +victor steinbeck 65618 +victor steinbeck 65618 +victor steinbeck 65618 +victor thompson 65548 +victor van buren 65664 +victor van buren 65664 +victor white 65548 +victor white 65548 +victor xylophone 65549 +victor xylophone 65549 +victor xylophone 65549 +victor xylophone 65549 +victor xylophone 65549 +victor young 65628 +victor zipper 65743 +wendy allen 65628 +wendy allen 65628 +wendy allen 65628 +wendy brown 65580 +wendy brown 65580 +wendy ellison 65545 +wendy ellison 65545 +wendy falkner 65595 +wendy falkner 65595 +wendy falkner 65595 +wendy garcia 65659 +wendy garcia 65659 +wendy garcia 65659 +wendy garcia 65659 +wendy hernandez 65650 +wendy ichabod 65730 +wendy king 65586 +wendy king 65586 +wendy king 65586 +wendy laertes 65566 +wendy laertes 65566 +wendy laertes 65566 +wendy miller 65582 +wendy miller 65582 +wendy nixon 65611 +wendy nixon 65611 +wendy ovid 65589 +wendy ovid 65589 +wendy polk 65656 +wendy polk 65656 +wendy quirinius 65766 +wendy quirinius 65766 +wendy robinson 65622 +wendy robinson 65622 +wendy robinson 65622 +wendy steinbeck 65612 +wendy thompson 65650 +wendy thompson 65650 +wendy underhill 65662 +wendy underhill 65662 +wendy underhill 65662 +wendy van buren 65680 +wendy van buren 65680 +wendy white 65705 +wendy xylophone 65687 +wendy xylophone 65687 +wendy young 65674 +wendy young 65674 +xavier allen 65611 +xavier allen 65611 +xavier allen 65611 +xavier brown 65600 +xavier brown 65600 +xavier brown 65600 +xavier carson 65731 +xavier carson 65731 +xavier davidson 65644 +xavier davidson 65644 +xavier davidson 65644 +xavier ellison 65541 +xavier ellison 65541 +xavier garcia 65672 +xavier hernandez 65541 +xavier hernandez 65541 +xavier hernandez 65541 +xavier ichabod 65597 +xavier ichabod 65597 +xavier johnson 65654 +xavier johnson 65654 +xavier king 65590 +xavier king 65590 +xavier laertes 65743 +xavier ovid 65788 +xavier polk 65587 +xavier polk 65587 +xavier polk 65587 +xavier polk 65587 +xavier quirinius 65599 +xavier quirinius 65599 +xavier quirinius 65599 +xavier quirinius 65599 +xavier thompson 65608 +xavier underhill 65710 +xavier white 65703 +xavier white 65703 +xavier xylophone 65572 +xavier zipper 65561 +yuri allen 65565 +yuri allen 65565 +yuri brown 65538 +yuri brown 65538 +yuri carson 65670 +yuri carson 65670 +yuri ellison 65570 +yuri ellison 65570 +yuri falkner 65658 +yuri falkner 65658 +yuri garcia 65639 +yuri hernandez 65706 +yuri johnson 65587 +yuri johnson 65587 +yuri johnson 65587 +yuri king 65721 +yuri laertes 65637 +yuri laertes 65637 +yuri nixon 65635 +yuri nixon 65635 +yuri polk 65607 +yuri polk 65607 +yuri polk 65607 +yuri quirinius 65544 +yuri quirinius 65544 +yuri quirinius 65544 +yuri steinbeck 65592 +yuri steinbeck 65592 +yuri thompson 65676 +yuri underhill 65718 +yuri underhill 65718 +yuri white 65659 +yuri xylophone 65714 +zach allen 65667 +zach brown 65559 +zach brown 65559 +zach brown 65559 +zach brown 65559 +zach brown 65559 +zach carson 65572 +zach ellison 65748 +zach falkner 65620 +zach falkner 65620 +zach garcia 65544 +zach garcia 65544 +zach garcia 65544 +zach garcia 65544 +zach ichabod 65599 +zach ichabod 65599 +zach king 65556 +zach king 65556 +zach king 65556 +zach miller 65584 +zach miller 65584 +zach miller 65584 +zach ovid 65578 +zach ovid 65578 +zach ovid 65578 +zach ovid 65578 +zach quirinius 65691 +zach robinson 65599 +zach steinbeck 65602 +zach steinbeck 65602 +zach thompson 65636 +zach thompson 65636 +zach underhill 65573 +zach white 65733 +zach xylophone 65542 +zach xylophone 65542 +zach young 65576 +zach zipper 65579 +zach zipper 65579 +zach zipper 65579 diff --git a/sql/hive/src/test/resources/golden/windowing_udaf.q (deterministic)-1-2bf20f39e6ffef258858f7943a974e7e b/sql/hive/src/test/resources/golden/windowing_udaf.q (deterministic)-1-2bf20f39e6ffef258858f7943a974e7e new file mode 100644 index 0000000000000..657e81a94f4c5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_udaf.q (deterministic)-1-2bf20f39e6ffef258858f7943a974e7e @@ -0,0 +1,1049 @@ + 9.220000267028809 + 43.72999954223633 + 89.52999877929688 +alice allen 2.7899999618530273 +alice allen 21.450000762939453 +alice allen 73.62999725341797 +alice brown 71.30999755859375 +alice carson 39.029998779296875 +alice davidson 70.3499984741211 +alice falkner 90.25 +alice garcia 48.45000076293945 +alice hernandez 88.16999816894531 +alice hernandez 90.55999755859375 +alice johnson 47.359999895095825 +alice king 19.139999389648438 +alice king 23.170000076293945 +alice king 52.22999954223633 +alice laertes 68.94999694824219 +alice laertes 69.52999877929688 +alice miller 68.95999908447266 +alice nixon 40.0 +alice nixon 48.150001525878906 +alice nixon 79.83000183105469 +alice ovid 9.039999961853027 +alice polk 62.900001525878906 +alice quirinius 37.13999938964844 +alice quirinius 62.29999923706055 +alice robinson 3.934999942779541 +alice robinson 56.099998474121094 +alice steinbeck 38.619998931884766 +alice steinbeck 63.40999794006348 +alice steinbeck 92.37000274658203 +alice underhill 93.17499923706055 +alice van buren 38.939998626708984 +alice xylophone 13.816667238871256 +alice xylophone 43.15999984741211 +alice xylophone 78.20999908447266 +alice zipper 26.43000030517578 +alice zipper 42.47999954223633 +alice zipper 89.93000030517578 +bob brown 8.069999694824219 +bob brown 70.93000030517578 +bob brown 93.08999633789062 +bob carson 50.09000015258789 +bob davidson 1.2899999618530273 +bob davidson 71.93000030517578 +bob davidson 74.72000122070312 +bob ellison 41.34000015258789 +bob ellison 56.584999084472656 +bob ellison 75.02999877929688 +bob ellison 80.30000305175781 +bob falkner 16.989999771118164 +bob garcia 4.460000038146973 +bob garcia 5.400000095367432 +bob garcia 43.46500015258789 +bob garcia 80.30000305175781 +bob garcia 87.56999969482422 +bob hernandez 55.51333363850912 +bob ichabod 82.55999755859375 +bob king 8.789999961853027 +bob king 12.539999961853027 +bob king 39.0099983215332 +bob laertes 0.7900000214576721 +bob laertes 10.670000076293945 +bob miller 61.91999816894531 +bob ovid 46.86000061035156 +bob ovid 62.849998474121094 +bob ovid 88.77999877929688 +bob ovid 97.08999633789062 +bob polk 7.980000019073486 +bob quirinius 48.09499931335449 +bob steinbeck 9.699999809265137 +bob van buren 33.66999816894531 +bob white 45.34000015258789 +bob white 45.349998474121094 +bob xylophone 27.6299991607666 +bob xylophone 29.359999656677246 +bob young 35.16999816894531 +bob zipper 3.819999933242798 +bob zipper 32.07500076293945 +bob zipper 52.63999938964844 +calvin allen 63.119998931884766 +calvin brown 28.110000610351562 +calvin brown 85.9000015258789 +calvin brown 90.19999694824219 +calvin carson 59.42500114440918 +calvin davidson 20.40500020980835 +calvin davidson 86.54000091552734 +calvin ellison 26.489999771118164 +calvin falkner 2.9700000286102295 +calvin falkner 56.040000915527344 +calvin falkner 56.33000183105469 +calvin falkner 80.5999984741211 +calvin falkner 93.61000061035156 +calvin falkner 94.30999755859375 +calvin garcia 41.849998474121094 +calvin hernandez 33.869998931884766 +calvin johnson 66.61000061035156 +calvin laertes 23.1299991607666 +calvin laertes 62.670000076293945 +calvin nixon 9.8100004196167 +calvin nixon 41.20000076293945 +calvin nixon 69.73999786376953 +calvin ovid 69.95999908447266 +calvin ovid 71.26000213623047 +calvin ovid 79.12000274658203 +calvin ovid 84.72000122070312 +calvin polk 65.72000122070312 +calvin quirinius 29.540000915527344 +calvin quirinius 60.98499870300293 +calvin robinson 40.439998626708984 +calvin steinbeck 15.220000267028809 +calvin steinbeck 22.850000381469727 +calvin steinbeck 52.06666819254557 +calvin thompson 8.90999984741211 +calvin thompson 89.22500228881836 +calvin underhill 59.70000076293945 +calvin van buren 34.209999084472656 +calvin van buren 64.0 +calvin white 35.864999771118164 +calvin white 90.69000244140625 +calvin xylophone 21.700000762939453 +calvin xylophone 25.420000076293945 +calvin xylophone 56.810001373291016 +calvin young 39.810001373291016 +calvin young 70.27999941507976 +calvin zipper 9.1899995803833 +calvin zipper 95.37999725341797 +david allen 51.005001068115234 +david allen 51.25 +david brown 32.56499910354614 +david brown 93.63999938964844 +david davidson 1.0800000429153442 +david davidson 54.17499876022339 +david davidson 62.720001220703125 +david davidson 74.1500015258789 +david ellison 47.689998626708984 +david ellison 62.587501525878906 +david ellison 85.2300033569336 +david hernandez 75.0833346048991 +david ichabod 35.763334115346275 +david ichabod 82.55000305175781 +david laertes 76.70999908447266 +david nixon 34.72999954223633 +david ovid 43.915000915527344 +david ovid 58.89999961853027 +david quirinius 23.5649995803833 +david quirinius 29.239999771118164 +david quirinius 79.97000122070312 +david robinson 37.703334172566734 +david robinson 70.55999755859375 +david thompson 43.619998931884766 +david underhill 49.719998359680176 +david underhill 60.36499881744385 +david underhill 60.46666717529297 +david van buren 25.500000476837158 +david van buren 50.27999973297119 +david white 45.189998626708984 +david xylophone 30.465000867843628 +david xylophone 30.465000867843628 +david xylophone 54.34000015258789 +david young 10.25 +david young 19.310000777244568 +ethan allen 23.790000518163044 +ethan brown 15.630000114440918 +ethan brown 22.93666648864746 +ethan brown 27.78000044822693 +ethan brown 52.19333346684774 +ethan brown 73.18000030517578 +ethan brown 82.30000305175781 +ethan carson 57.635000228881836 +ethan ellison 0.2800000011920929 +ethan ellison 81.47000122070312 +ethan falkner 50.02000045776367 +ethan falkner 59.43000030517578 +ethan garcia 26.44499921798706 +ethan hernandez 32.30333264668783 +ethan johnson 90.05000305175781 +ethan king 4.349999904632568 +ethan laertes 54.75 +ethan laertes 54.87999868392944 +ethan laertes 59.209999084472656 +ethan laertes 75.31500053405762 +ethan laertes 76.94499969482422 +ethan laertes 80.4749984741211 +ethan laertes 95.06999969482422 +ethan miller 25.3700008392334 +ethan nixon 50.88999938964844 +ethan ovid 58.4950008392334 +ethan polk 2.3499999046325684 +ethan polk 21.31999969482422 +ethan polk 23.440000534057617 +ethan polk 59.869998931884766 +ethan quirinius 13.650000214576721 +ethan quirinius 70.94500160217285 +ethan quirinius 88.18000030517578 +ethan robinson 67.94000244140625 +ethan robinson 80.58500289916992 +ethan underhill 55.630001068115234 +ethan van buren 37.85499954223633 +ethan white 58.70666631062826 +ethan white 63.41999816894531 +ethan xylophone 57.11000061035156 +ethan zipper 5.914999961853027 +ethan zipper 97.51000213623047 +fred davidson 26.435000896453857 +fred davidson 28.144000816345216 +fred davidson 78.30999755859375 +fred ellison 46.65999984741211 +fred ellison 65.40666516621907 +fred ellison 71.98499870300293 +fred falkner 25.144999980926514 +fred falkner 37.62000068028768 +fred falkner 75.125 +fred hernandez 55.9900016784668 +fred ichabod 37.06999969482422 +fred ichabod 75.83499908447266 +fred johnson 96.08999633789062 +fred king 10.220000267028809 +fred king 68.40500068664551 +fred laertes 42.68750047683716 +fred miller 70.02999877929688 +fred nixon 30.589999198913574 +fred nixon 32.71666653951009 +fred nixon 70.5199966430664 +fred nixon 93.02999877929688 +fred polk 23.959999084472656 +fred polk 47.31999969482422 +fred polk 63.97999954223633 +fred polk 90.12000274658203 +fred quirinius 15.300000190734863 +fred quirinius 27.40999984741211 +fred robinson 64.42000007629395 +fred steinbeck 21.239999771118164 +fred steinbeck 21.30000066757202 +fred steinbeck 65.44333394368489 +fred underhill 85.36999893188477 +fred van buren 22.37499976158142 +fred van buren 44.49000096321106 +fred van buren 45.94999980926514 +fred van buren 56.88999938964844 +fred white 42.329999923706055 +fred young 46.79999923706055 +fred young 72.69999885559082 +fred zipper 50.14000129699707 +gabriella allen 28.040000438690186 +gabriella allen 79.64500045776367 +gabriella brown 26.164999961853027 +gabriella brown 84.83000183105469 +gabriella carson 42.7599983215332 +gabriella davidson 18.946666717529297 +gabriella ellison 48.08000183105469 +gabriella ellison 71.54000091552734 +gabriella falkner 31.609999974568684 +gabriella falkner 51.720001220703125 +gabriella falkner 87.61000061035156 +gabriella garcia 43.0099983215332 +gabriella hernandez 62.22666517893473 +gabriella hernandez 92.9800033569336 +gabriella ichabod 10.729999542236328 +gabriella ichabod 26.639999389648438 +gabriella ichabod 66.36000061035156 +gabriella ichabod 71.12999725341797 +gabriella ichabod 83.76666514078777 +gabriella king 20.670000076293945 +gabriella king 62.89999961853027 +gabriella laertes 62.62499809265137 +gabriella miller 26.043334086736042 +gabriella ovid 44.78000068664551 +gabriella ovid 92.4000015258789 +gabriella polk 35.68000030517578 +gabriella polk 90.22500228881836 +gabriella steinbeck 46.45000076293945 +gabriella steinbeck 66.86999893188477 +gabriella thompson 72.34500122070312 +gabriella thompson 75.73666636149089 +gabriella thompson 94.25 +gabriella van buren 38.935001373291016 +gabriella van buren 48.349998474121094 +gabriella white 55.18000030517578 +gabriella young 32.16333262125651 +gabriella young 59.709999084472656 +gabriella zipper 36.2599983215332 +gabriella zipper 91.62999725341797 +holly allen 63.435001373291016 +holly brown 68.51666641235352 +holly brown 86.08666737874348 +holly falkner 83.82666524251302 +holly hernandez 21.190000534057617 +holly hernandez 24.790000915527344 +holly hernandez 28.649999300638836 +holly hernandez 50.22999954223633 +holly ichabod 48.86333433787028 +holly ichabod 73.77000045776367 +holly ichabod 82.91499710083008 +holly johnson 23.447500228881836 +holly johnson 64.36000061035156 +holly johnson 65.62000274658203 +holly king 35.34499979019165 +holly king 42.310001373291016 +holly laertes 70.59666697184245 +holly miller 35.86000061035156 +holly nixon 43.82499885559082 +holly nixon 78.80749893188477 +holly polk 30.365000247955322 +holly polk 75.96499919891357 +holly robinson 82.70499801635742 +holly thompson 0.07999999821186066 +holly thompson 65.52499842643738 +holly thompson 86.69000244140625 +holly underhill 42.54999923706055 +holly underhill 53.02000045776367 +holly underhill 56.13333384195963 +holly underhill 65.84000015258789 +holly van buren 48.893333435058594 +holly white 26.5633331934611 +holly white 40.5 +holly xylophone 49.55666716893514 +holly young 41.698571750095915 +holly young 75.20999908447266 +holly zipper 79.72999827067058 +holly zipper 81.08666610717773 +irene allen 29.75999927520752 +irene brown 28.596666653951008 +irene brown 47.189998626708984 +irene brown 49.46666622161865 +irene carson 86.64999898274739 +irene ellison 38.255001068115234 +irene ellison 45.71333376566569 +irene falkner 22.079999923706055 +irene falkner 83.44666544596355 +irene garcia 38.93499994277954 +irene garcia 42.84666601816813 +irene garcia 58.43000030517578 +irene ichabod 60.7079984664917 +irene ichabod 64.58000183105469 +irene johnson 26.165000438690186 +irene laertes 21.02999997138977 +irene laertes 40.04499816894531 +irene laertes 47.04333241780599 +irene miller 65.44000244140625 +irene nixon 46.03999996185303 +irene nixon 46.96666653951009 +irene nixon 67.09499931335449 +irene ovid 35.130001068115234 +irene ovid 42.535000801086426 +irene ovid 79.75 +irene polk 0.9800000190734863 +irene polk 35.17500114440918 +irene polk 48.94666576385498 +irene polk 49.6 +irene polk 51.885000228881836 +irene quirinius 33.78000005086263 +irene quirinius 42.610000133514404 +irene quirinius 53.4800001780192 +irene robinson 92.19499969482422 +irene steinbeck 73.28000068664551 +irene thompson 46.27375066280365 +irene underhill 24.75 +irene underhill 57.349998474121094 +irene van buren 50.8799991607666 +irene van buren 74.5625 +irene xylophone 83.53499984741211 +jessica brown 26.185000479221344 +jessica carson 56.22999954223633 +jessica carson 62.20000076293945 +jessica carson 62.2400016784668 +jessica davidson 50.01666768391927 +jessica davidson 63.59499931335449 +jessica davidson 69.26666514078777 +jessica davidson 94.53333282470703 +jessica ellison 11.180000305175781 +jessica ellison 64.2060001373291 +jessica falkner 61.57333437601725 +jessica garcia 38.55250036716461 +jessica garcia 57.00999975204468 +jessica ichabod 32.63250035047531 +jessica johnson 9.5600004196167 +jessica johnson 51.959999084472656 +jessica miller 77.83999633789062 +jessica nixon 40.72249960899353 +jessica nixon 90.06999969482422 +jessica ovid 37.45250064134598 +jessica ovid 59.68000030517578 +jessica polk 49.68000030517578 +jessica quirinius 25.65750002861023 +jessica quirinius 37.64200019836426 +jessica quirinius 54.25500011444092 +jessica quirinius 58.019999186197914 +jessica robinson 42.66333484649658 +jessica thompson 30.40666675567627 +jessica thompson 43.87500023841858 +jessica underhill 43.33333269755045 +jessica underhill 45.639999866485596 +jessica underhill 57.584999084472656 +jessica van buren 67.00000047683716 +jessica white 6.170000106096268 +jessica white 63.32500076293945 +jessica white 65.1450023651123 +jessica white 73.93000030517578 +jessica white 96.62000274658203 +jessica xylophone 69.87500190734863 +jessica young 11.1899995803833 +jessica young 43.369998931884766 +jessica zipper 42.43833335240682 +jessica zipper 46.7450008392334 +jessica zipper 56.97999954223633 +katie allen 55.47666549682617 +katie brown 31.699999809265137 +katie davidson 93.22000122070312 +katie ellison 48.31999933719635 +katie ellison 64.08499892552693 +katie falkner 51.665000915527344 +katie garcia 57.71000099182129 +katie garcia 61.21000051498413 +katie hernandez 41.150000381469724 +katie ichabod 44.243333180745445 +katie ichabod 51.800000508626304 +katie ichabod 69.18799896240235 +katie king 39.83000183105469 +katie king 46.80333296457926 +katie king 51.85000038146973 +katie miller 31.399999618530273 +katie miller 74.77999877929688 +katie nixon 23.190000534057617 +katie ovid 67.94500160217285 +katie polk 26.62750005722046 +katie polk 33.9350004196167 +katie robinson 13.890000343322754 +katie van buren 44.434998512268066 +katie van buren 65.41999816894531 +katie white 37.96500015258789 +katie white 59.223333517710365 +katie xylophone 39.30000019073486 +katie young 36.660000801086426 +katie young 67.78333282470703 +katie young 72.76666577657063 +katie zipper 23.766667087872822 +katie zipper 58.75 +luke allen 50.959999084472656 +luke allen 53.36666742960612 +luke allen 54.63249969482422 +luke allen 57.670000076293945 +luke allen 70.39500045776367 +luke brown 49.595000982284546 +luke davidson 7.050000190734863 +luke davidson 18.87000036239624 +luke ellison 16.25 +luke ellison 32.9519996881485 +luke ellison 71.93500137329102 +luke falkner 21.71999979019165 +luke falkner 31.81250023841858 +luke garcia 18.65499973297119 +luke garcia 41.2300001780192 +luke ichabod 41.25750005245209 +luke ichabod 73.55000114440918 +luke johnson 31.670000076293945 +luke johnson 32.84499979019165 +luke johnson 39.54500102996826 +luke laertes 11.819999694824219 +luke laertes 21.184999227523804 +luke laertes 21.993332862854004 +luke laertes 26.696666717529297 +luke laertes 45.9900016784668 +luke miller 52.350000858306885 +luke ovid 23.804999828338623 +luke ovid 64.30000305175781 +luke polk 41.02499961853027 +luke polk 58.4566650390625 +luke quirinius 40.41999816894531 +luke robinson 48.559998750686646 +luke robinson 56.76499938964844 +luke thompson 78.04333368937175 +luke underhill 34.0166662534078 +luke underhill 47.28999996185303 +luke underhill 59.32000160217285 +luke van buren 59.91999944051107 +luke white 74.19599990844726 +luke xylophone 64.77999925613403 +luke zipper 30.434999465942383 +mike allen 30.539999961853027 +mike brown 69.86833318074544 +mike carson 30.25333309173584 +mike carson 61.33799934387207 +mike carson 89.375 +mike davidson 32.55333391825358 +mike davidson 66.74333318074544 +mike ellison 35.905999755859376 +mike ellison 39.82499885559082 +mike ellison 58.56399993896484 +mike ellison 64.52999877929688 +mike ellison 66.93749856948853 +mike falkner 48.53750002384186 +mike garcia 51.02999973297119 +mike garcia 67.93000030517578 +mike garcia 70.8499984741211 +mike hernandez 37.900001525878906 +mike hernandez 59.45000076293945 +mike ichabod 64.7699966430664 +mike king 36.17800045013428 +mike king 41.69500136375427 +mike king 49.57000017166138 +mike king 59.654998779296875 +mike king 71.57000122070312 +mike king 78.50999927520752 +mike miller 29.570000171661377 +mike nixon 45.029999828338624 +mike nixon 48.429999669392906 +mike polk 30.864000129699708 +mike polk 46.95499873161316 +mike polk 79.55500030517578 +mike quirinius 85.0699971516927 +mike steinbeck 24.267500042915344 +mike steinbeck 43.52500021457672 +mike steinbeck 61.426666577657066 +mike steinbeck 68.46000022888184 +mike van buren 27.639999389648438 +mike van buren 56.16333134969076 +mike white 34.8924994468689 +mike white 43.5566660563151 +mike white 53.689998626708984 +mike white 77.54499864578247 +mike young 34.3319993019104 +mike young 52.8100004196167 +mike young 55.64333359400431 +mike zipper 56.86666742960612 +mike zipper 63.3149995803833 +mike zipper 83.91999816894531 +nick allen 57.086001586914065 +nick allen 60.15400094985962 +nick brown 42.939998626708984 +nick davidson 63.07499885559082 +nick ellison 45.34000015258789 +nick ellison 65.88500022888184 +nick falkner 41.87999868392944 +nick falkner 64.05666732788086 +nick garcia 34.34499979019165 +nick garcia 51.08666737874349 +nick garcia 62.88600044250488 +nick ichabod 20.253333409627277 +nick ichabod 53.635000228881836 +nick ichabod 77.36000061035156 +nick johnson 20.114999175071716 +nick johnson 81.91666666666667 +nick laertes 91.56666819254558 +nick miller 71.5500005086263 +nick nixon 77.04249954223633 +nick ovid 74.62666702270508 +nick polk 39.27500009536743 +nick quirinius 60.79499816894531 +nick quirinius 67.44999694824219 +nick robinson 31.672499418258667 +nick robinson 57.66999816894531 +nick steinbeck 59.15999984741211 +nick thompson 18.88666645685832 +nick underhill 43.009998893737794 +nick van buren 34.720000902811684 +nick xylophone 75.3499984741211 +nick young 0.27000001072883606 +nick young 47.813334147135414 +nick zipper 46.22333272298177 +nick zipper 52.54333209991455 +oscar allen 37.396666844685875 +oscar brown 13.100000381469727 +oscar carson 31.91333230336507 +oscar carson 41.77333331108093 +oscar carson 57.3149995803833 +oscar carson 73.59500122070312 +oscar carson 95.44000244140625 +oscar davidson 75.18000030517578 +oscar ellison 34.04499959945679 +oscar ellison 34.04499959945679 +oscar falkner 61.72000050544739 +oscar garcia 67.4800033569336 +oscar hernandez 41.63333400090536 +oscar hernandez 47.93999986648559 +oscar ichabod 45.839999516805015 +oscar ichabod 68.62000274658203 +oscar ichabod 72.18249797821045 +oscar ichabod 76.69000244140625 +oscar johnson 23.880000114440918 +oscar johnson 65.04000091552734 +oscar king 36.69500017166138 +oscar king 49.7049994468689 +oscar king 67.98399925231934 +oscar laertes 43.616665522257485 +oscar laertes 44.755000591278076 +oscar laertes 45.26666831970215 +oscar laertes 53.710001945495605 +oscar nixon 36.56999937693278 +oscar ovid 45.89999961853027 +oscar ovid 46.93999934196472 +oscar ovid 55.277999591827395 +oscar polk 42.31999969482422 +oscar polk 63.900001525878906 +oscar quirinius 63.81500053405762 +oscar quirinius 66.28428527287075 +oscar quirinius 70.24000295003255 +oscar quirinius 81.26249980926514 +oscar robinson 11.34000015258789 +oscar robinson 47.845001220703125 +oscar robinson 59.74333349863688 +oscar robinson 63.346666971842446 +oscar steinbeck 42.49999976158142 +oscar thompson 38.23500061035156 +oscar thompson 51.469999154408775 +oscar thompson 60.029999542236325 +oscar thompson 63.079999923706055 +oscar underhill 66.97666676839192 +oscar van buren 24.085000872612 +oscar van buren 61.880001068115234 +oscar van buren 72.9533322652181 +oscar white 44.72333272298177 +oscar white 46.60999870300293 +oscar white 54.7599983215332 +oscar white 60.85500144958496 +oscar xylophone 34.946666399637856 +oscar xylophone 39.8299994468689 +oscar xylophone 57.119998931884766 +oscar zipper 28.499999046325684 +oscar zipper 47.46750068664551 +oscar zipper 59.1933339436849 +priscilla brown 47.40400066375732 +priscilla brown 77.1479995727539 +priscilla brown 80.5199966430664 +priscilla carson 7.960000038146973 +priscilla carson 28.480000153183937 +priscilla carson 45.92750144004822 +priscilla ichabod 38.95666758219401 +priscilla ichabod 62.32999928792318 +priscilla johnson 44.04499912261963 +priscilla johnson 50.53750038146973 +priscilla johnson 55.98333485921224 +priscilla johnson 59.64499855041504 +priscilla johnson 89.1500015258789 +priscilla king 50.44666735331217 +priscilla nixon 44.32222270965576 +priscilla nixon 45.267999792099 +priscilla ovid 44.78333361943563 +priscilla ovid 52.72999954223633 +priscilla polk 34.89399948120117 +priscilla quirinius 35.609999895095825 +priscilla thompson 35.16249918937683 +priscilla underhill 68.22000122070312 +priscilla underhill 73.97200012207031 +priscilla van buren 50.47000026702881 +priscilla van buren 51.39500045776367 +priscilla van buren 53.541999435424806 +priscilla white 50.47599992752075 +priscilla xylophone 0.15000000596046448 +priscilla xylophone 41.106666485468544 +priscilla xylophone 63.9574990272522 +priscilla young 0.2900000065565109 +priscilla young 19.866666316986084 +priscilla zipper 32.084999322891235 +priscilla zipper 43.90333366394043 +quinn allen 47.90333382288615 +quinn allen 83.33000183105469 +quinn brown 24.280000686645508 +quinn brown 53.98666508992513 +quinn brown 66.82500171661377 +quinn davidson 40.666666666666664 +quinn davidson 54.095001220703125 +quinn davidson 79.78333282470703 +quinn davidson 92.13000106811523 +quinn ellison 52.714999198913574 +quinn ellison 63.352500438690186 +quinn garcia 20.19000056385994 +quinn garcia 54.60000038146973 +quinn garcia 59.010000824928284 +quinn garcia 68.98999881744385 +quinn ichabod 48.60499930381775 +quinn king 61.27333450317383 +quinn king 81.46000289916992 +quinn laertes 32.08000040054321 +quinn laertes 44.45666694641113 +quinn laertes 49.85499858856201 +quinn nixon 72.2471422467913 +quinn ovid 34.423333168029785 +quinn quirinius 53.165000915527344 +quinn robinson 32.624999046325684 +quinn steinbeck 24.802499771118164 +quinn steinbeck 55.477500915527344 +quinn thompson 50.500000381469725 +quinn thompson 55.68600006103516 +quinn underhill 39.66600060462952 +quinn underhill 41.47666676839193 +quinn underhill 56.580000162124634 +quinn van buren 49.40333207448324 +quinn young 55.59000142415365 +quinn zipper 11.359999974568685 +quinn zipper 48.45000123977661 +rachel allen 45.940001249313354 +rachel allen 85.97999954223633 +rachel brown 33.01999984184901 +rachel brown 34.08250021934509 +rachel brown 37.999999046325684 +rachel brown 41.75000019868215 +rachel brown 53.679999669392906 +rachel carson 43.32400016784668 +rachel carson 66.2233320871989 +rachel davidson 14.220000267028809 +rachel ellison 17.549999833106995 +rachel falkner 56.883334159851074 +rachel falkner 57.5199998219808 +rachel falkner 58.80666637420654 +rachel falkner 70.69428443908691 +rachel johnson 36.22499990463257 +rachel king 50.970001220703125 +rachel king 83.53750133514404 +rachel laertes 42.29857151848929 +rachel laertes 71.65999984741211 +rachel ovid 42.25333329041799 +rachel ovid 47.01749947667122 +rachel polk 64.90333239237468 +rachel quirinius 53.2624990940094 +rachel robinson 40.712857246398926 +rachel robinson 53.092498898506165 +rachel robinson 64.94999694824219 +rachel thompson 24.555000439286232 +rachel thompson 31.460000038146973 +rachel thompson 46.804000282287596 +rachel underhill 47.22333272298177 +rachel white 39.87999979654948 +rachel white 41.83428575311388 +rachel young 75.7966677347819 +rachel zipper 45.794999519983925 +rachel zipper 56.909999179840085 +sarah carson 24.576666196187336 +sarah carson 36.33750060200691 +sarah carson 43.65749907493591 +sarah ellison 37.054999351501465 +sarah falkner 48.58285754067557 +sarah falkner 62.36500072479248 +sarah garcia 33.38000011444092 +sarah garcia 35.513333002726235 +sarah garcia 64.31333287556966 +sarah ichabod 36.10599975585937 +sarah ichabod 45.830000162124634 +sarah johnson 26.464999675750732 +sarah johnson 40.9300012588501 +sarah johnson 43.44000196456909 +sarah johnson 64.24333318074544 +sarah king 49.06999909877777 +sarah king 63.01333363850912 +sarah miller 41.709999084472656 +sarah ovid 63.682499408721924 +sarah robinson 39.196666399637856 +sarah robinson 66.88999938964844 +sarah steinbeck 66.89000034332275 +sarah white 41.42599945068359 +sarah white 52.95249938964844 +sarah xylophone 68.31999969482422 +sarah young 35.92750024795532 +sarah zipper 53.697500705718994 +tom brown 38.37000020345052 +tom brown 44.68000049591065 +tom carson 27.994999766349792 +tom carson 54.25250005722046 +tom carson 62.790000915527344 +tom davidson 38.679999113082886 +tom ellison 33.68600053787232 +tom ellison 46.00666618347168 +tom ellison 67.79666646321614 +tom falkner 55.61800079345703 +tom falkner 58.82500012715658 +tom hernandez 50.52250051498413 +tom hernandez 50.52250051498413 +tom ichabod 24.98399963378906 +tom johnson 34.83750009536743 +tom johnson 73.72399978637695 +tom king 69.98000106811523 +tom laertes 41.97285750934056 +tom laertes 70.40333429972331 +tom miller 43.885000586509705 +tom miller 57.10500144958496 +tom miller 76.20499992370605 +tom nixon 62.43000030517578 +tom ovid 38.096666971842446 +tom polk 51.26750087738037 +tom polk 68.22666676839192 +tom quirinius 37.720001220703125 +tom quirinius 53.20399913787842 +tom robinson 43.44333299001058 +tom robinson 54.637142998831614 +tom robinson 59.34250068664551 +tom robinson 99.1500015258789 +tom steinbeck 51.883334477742515 +tom van buren 28.380000829696655 +tom van buren 35.64999930063883 +tom van buren 54.59000015258789 +tom white 51.970001220703125 +tom young 44.7319995880127 +tom young 53.894999980926514 +tom zipper 55.44000116984049 +ulysses brown 48.72666708628336 +ulysses carson 38.742000579833984 +ulysses carson 45.513333002726235 +ulysses carson 48.75249934196472 +ulysses carson 74.64600067138672 +ulysses davidson 63.20857129778181 +ulysses ellison 68.52666759490967 +ulysses garcia 58.77250051498413 +ulysses hernandez 32.371999168395995 +ulysses hernandez 50.57000102996826 +ulysses hernandez 61.39999961853027 +ulysses ichabod 19.1299991607666 +ulysses ichabod 83.06666692097981 +ulysses johnson 51.485000451405845 +ulysses king 46.98333422342936 +ulysses laertes 29.046666741371155 +ulysses laertes 32.88599967956543 +ulysses laertes 60.12399845123291 +ulysses miller 44.552857535226 +ulysses miller 71.39249873161316 +ulysses nixon 51.300000286102296 +ulysses ovid 29.360000610351562 +ulysses polk 40.74399948120117 +ulysses polk 48.9800017674764 +ulysses polk 57.86249828338623 +ulysses polk 81.21333312988281 +ulysses quirinius 68.41500091552734 +ulysses robinson 69.53999853134155 +ulysses steinbeck 44.61833381652832 +ulysses steinbeck 48.362499713897705 +ulysses thompson 45.063334465026855 +ulysses underhill 30.829999764760334 +ulysses underhill 41.43857192993164 +ulysses underhill 44.08333269755045 +ulysses underhill 55.470001220703125 +ulysses underhill 58.9471435546875 +ulysses underhill 68.1900007724762 +ulysses underhill 78.83333333333333 +ulysses van buren 72.38428633553642 +ulysses white 36.17250043153763 +ulysses white 39.084000778198245 +ulysses xylophone 27.519999504089355 +ulysses xylophone 47.65999937057495 +ulysses xylophone 50.29999923706055 +ulysses young 23.308333079020183 +ulysses young 34.6339994430542 +ulysses young 88.06999969482422 +victor allen 49.43800010681152 +victor allen 56.7299998147147 +victor brown 40.80600037574768 +victor brown 63.5024995803833 +victor brown 71.03500080108643 +victor brown 81.71999931335449 +victor davidson 44.70333290100098 +victor davidson 59.070000076293944 +victor davidson 67.27199935913086 +victor ellison 31.28999964396159 +victor ellison 42.54999923706055 +victor hernandez 44.41333452860514 +victor hernandez 47.20249938964844 +victor hernandez 47.73333215713501 +victor hernandez 51.04999923706055 +victor hernandez 59.2399995803833 +victor johnson 54.868000626564026 +victor johnson 55.22999954223633 +victor johnson 57.41000175476074 +victor king 38.27999997138977 +victor king 49.993333180745445 +victor laertes 40.63500006993612 +victor laertes 80.5999984741211 +victor miller 71.00000190734863 +victor nixon 38.393332640329994 +victor nixon 52.920000076293945 +victor ovid 53.260000228881836 +victor polk 3.0 +victor quirinius 57.81666644414266 +victor quirinius 59.39999923706055 +victor robinson 24.614999771118164 +victor robinson 74.5049991607666 +victor steinbeck 28.862000381946565 +victor steinbeck 36.61000006539481 +victor steinbeck 43.09000015258789 +victor thompson 42.67599925994873 +victor van buren 44.669999877611794 +victor van buren 45.121999740600586 +victor white 53.67999887466431 +victor white 54.45000012715658 +victor xylophone 12.160000324249268 +victor xylophone 26.0 +victor xylophone 31.769999821980793 +victor xylophone 52.31499926249186 +victor xylophone 69.2899996439616 +victor young 64.25833320617676 +victor zipper 65.24999904632568 +wendy allen 34.04999955495199 +wendy allen 36.88199939727783 +wendy allen 44.96000012755394 +wendy brown 45.97833283742269 +wendy brown 52.73857225690569 +wendy ellison 42.91333246231079 +wendy ellison 53.56000073750814 +wendy falkner 47.602500438690186 +wendy falkner 64.9099988937378 +wendy falkner 77.5999984741211 +wendy garcia 37.38571425846645 +wendy garcia 48.76666768391927 +wendy garcia 53.225000858306885 +wendy garcia 63.93999926249186 +wendy hernandez 36.195000648498535 +wendy ichabod 10.56499981880188 +wendy king 37.57500076293945 +wendy king 53.44333457946777 +wendy king 56.319997787475586 +wendy laertes 38.39249920845032 +wendy laertes 60.19999885559082 +wendy laertes 65.30624961853027 +wendy miller 44.273332595825195 +wendy miller 53.5675014257431 +wendy nixon 54.995998764038085 +wendy nixon 64.28250026702881 +wendy ovid 43.80499863624573 +wendy ovid 61.64600105285645 +wendy polk 26.784999758005142 +wendy polk 35.21599998474121 +wendy quirinius 28.75666618347168 +wendy quirinius 60.70000092188517 +wendy robinson 42.5799994468689 +wendy robinson 42.90799944400787 +wendy robinson 43.426000237464905 +wendy steinbeck 43.42333388328552 +wendy thompson 40.352857317243306 +wendy thompson 75.93666712443034 +wendy underhill 24.459999561309814 +wendy underhill 33.440000693003334 +wendy underhill 45.51625019311905 +wendy van buren 43.30333296457926 +wendy van buren 65.58666737874348 +wendy white 39.015000104904175 +wendy xylophone 42.42500034968058 +wendy xylophone 53.981666247049965 +wendy young 27.929999828338623 +wendy young 59.609999656677246 +xavier allen 49.24500061571598 +xavier allen 67.39000034332275 +xavier allen 70.29800033569336 +xavier brown 19.772500306367874 +xavier brown 58.87000111171177 +xavier brown 74.6200008392334 +xavier carson 47.3199987411499 +xavier carson 61.52250027656555 +xavier davidson 48.41999936103821 +xavier davidson 52.70666758219401 +xavier davidson 52.78166747093201 +xavier ellison 41.84999942779541 +xavier ellison 62.80200090408325 +xavier garcia 35.0600004568696 +xavier hernandez 47.45200023651123 +xavier hernandez 49.676666259765625 +xavier hernandez 53.446667432785034 +xavier ichabod 56.70625042915344 +xavier ichabod 60.54799928665161 +xavier johnson 41.16333262125651 +xavier johnson 53.85333331425985 +xavier king 42.72800064086914 +xavier king 66.05333455403645 +xavier laertes 38.47999954223633 +xavier ovid 48.89250057935715 +xavier polk 37.05500018596649 +xavier polk 46.82666703065237 +xavier polk 55.385000705718994 +xavier polk 55.65000057220459 +xavier quirinius 59.62499964237213 +xavier quirinius 60.055998992919925 +xavier quirinius 62.52000045776367 +xavier quirinius 65.2933349609375 +xavier thompson 40.244998931884766 +xavier underhill 26.27800006866455 +xavier white 47.8671429497855 +xavier white 63.38428551810129 +xavier xylophone 49.072500228881836 +xavier zipper 8.204999923706055 +yuri allen 53.61250066757202 +yuri allen 64.86833254496257 +yuri brown 46.57500044504801 +yuri brown 66.75250005722046 +yuri carson 39.40750050544739 +yuri carson 49.01600036621094 +yuri ellison 27.49000017642975 +yuri ellison 70.5933329264323 +yuri falkner 47.23285675048828 +yuri falkner 62.807999801635745 +yuri garcia 43.967499017715454 +yuri hernandez 31.94000039100647 +yuri johnson 21.40666739145915 +yuri johnson 34.02333414554596 +yuri johnson 65.7750015258789 +yuri king 49.47333272298177 +yuri laertes 42.070000648498535 +yuri laertes 60.7549991607666 +yuri nixon 49.87142838750567 +yuri nixon 59.945000330607094 +yuri polk 37.56249952316284 +yuri polk 47.583333333333336 +yuri polk 72.60888735453288 +yuri quirinius 18.62000060081482 +yuri quirinius 51.217501401901245 +yuri quirinius 67.24000072479248 +yuri steinbeck 55.757999420166016 +yuri steinbeck 75.87999725341797 +yuri thompson 36.93499946594238 +yuri underhill 51.533334732055664 +yuri underhill 62.31888887617323 +yuri white 44.34999983651297 +yuri xylophone 25.117499828338623 +zach allen 25.92333350578944 +zach brown 38.3799991607666 +zach brown 47.404998779296875 +zach brown 54.30600090026856 +zach brown 58.970001220703125 +zach brown 65.22499942779541 +zach carson 60.783999633789065 +zach ellison 36.211428437914165 +zach falkner 41.225714683532715 +zach falkner 65.99499940872192 +zach garcia 42.8885714326586 +zach garcia 46.8870005607605 +zach garcia 47.5049991607666 +zach garcia 66.09399967193603 +zach ichabod 40.10166613260905 +zach ichabod 53.16749954223633 +zach king 39.137500405311584 +zach king 48.2825003862381 +zach king 61.18999965985616 +zach miller 44.82800054550171 +zach miller 48.52428477151053 +zach miller 53.593332608540855 +zach ovid 35.19399921447039 +zach ovid 38.35833342870077 +zach ovid 43.87200012207031 +zach ovid 83.01999918619792 +zach quirinius 42.638333320617676 +zach robinson 82.04999923706055 +zach steinbeck 55.86599960327148 +zach steinbeck 67.81428473336356 +zach thompson 29.303333282470703 +zach thompson 46.48999913533529 +zach underhill 48.681429045540945 +zach white 66.60250091552734 +zach xylophone 41.875 +zach xylophone 57.2416664759318 +zach young 73.5999984741211 +zach zipper 58.1480016708374 +zach zipper 60.1825008392334 +zach zipper 62.794999837875366 diff --git a/sql/hive/src/test/resources/golden/windowing_udaf.q (deterministic)-2-16239d2b069789ba99fbac50c4f0724f b/sql/hive/src/test/resources/golden/windowing_udaf.q (deterministic)-2-16239d2b069789ba99fbac50c4f0724f new file mode 100644 index 0000000000000..6cfa5ad413fa4 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_udaf.q (deterministic)-2-16239d2b069789ba99fbac50c4f0724f @@ -0,0 +1,1049 @@ + 65560.0 + 65718.0 + 65740.0 +alice allen 65662.0 +alice allen 65720.0 +alice allen 65758.0 +alice brown 65696.0 +alice carson 65559.0 +alice davidson 65547.0 +alice falkner 65669.0 +alice garcia 65613.0 +alice hernandez 65737.0 +alice hernandez 65784.0 +alice johnson 65739.0 +alice king 65660.0 +alice king 65734.0 +alice king 65738.0 +alice laertes 65669.0 +alice laertes 65671.0 +alice miller 65590.0 +alice nixon 65586.0 +alice nixon 65595.0 +alice nixon 65604.0 +alice ovid 65737.0 +alice polk 65548.0 +alice quirinius 65636.0 +alice quirinius 65728.0 +alice robinson 65606.0 +alice robinson 65789.0 +alice steinbeck 65578.0 +alice steinbeck 65673.0 +alice steinbeck 65786.0 +alice underhill 65750.0 +alice van buren 65562.0 +alice xylophone 65578.0 +alice xylophone 65585.0 +alice xylophone 65599.0 +alice zipper 65553.0 +alice zipper 65662.0 +alice zipper 65766.0 +bob brown 65584.0 +bob brown 65777.0 +bob brown 65783.0 +bob carson 65713.0 +bob davidson 65664.0 +bob davidson 65693.0 +bob davidson 65768.0 +bob ellison 65591.0 +bob ellison 65624.0 +bob ellison 65721.0 +bob ellison 65760.0 +bob falkner 65789.0 +bob garcia 65585.0 +bob garcia 65598.0 +bob garcia 65673.0 +bob garcia 65754.0 +bob garcia 65782.0 +bob hernandez 65557.0 +bob ichabod 65549.0 +bob king 65715.0 +bob king 65757.0 +bob king 65783.0 +bob laertes 65602.0 +bob laertes 65663.0 +bob miller 65608.0 +bob ovid 65564.0 +bob ovid 65619.0 +bob ovid 65686.0 +bob ovid 65726.0 +bob polk 65594.0 +bob quirinius 65700.0 +bob steinbeck 65637.0 +bob van buren 65778.0 +bob white 65543.0 +bob white 65605.0 +bob xylophone 65574.0 +bob xylophone 65666.0 +bob young 65556.0 +bob zipper 65559.0 +bob zipper 65633.0 +bob zipper 65739.0 +calvin allen 65669.0 +calvin brown 65537.0 +calvin brown 65580.0 +calvin brown 65677.0 +calvin carson 65637.0 +calvin davidson 65541.0 +calvin davidson 65564.0 +calvin ellison 65667.0 +calvin falkner 65573.0 +calvin falkner 65596.0 +calvin falkner 65738.0 +calvin falkner 65762.0 +calvin falkner 65778.0 +calvin falkner 65784.0 +calvin garcia 65664.0 +calvin hernandez 65578.0 +calvin johnson 65731.0 +calvin laertes 65570.0 +calvin laertes 65684.0 +calvin nixon 65654.0 +calvin nixon 65724.0 +calvin nixon 65749.0 +calvin ovid 65554.0 +calvin ovid 65643.0 +calvin ovid 65663.0 +calvin ovid 65715.0 +calvin polk 65731.0 +calvin quirinius 65741.0 +calvin quirinius 65769.0 +calvin robinson 65581.0 +calvin steinbeck 65680.0 +calvin steinbeck 65762.0 +calvin steinbeck 65779.0 +calvin thompson 65560.0 +calvin thompson 65640.0 +calvin underhill 65732.0 +calvin van buren 65552.0 +calvin van buren 65771.0 +calvin white 65553.0 +calvin white 65561.0 +calvin xylophone 65575.0 +calvin xylophone 65614.5 +calvin xylophone 65713.0 +calvin young 65574.0 +calvin young 65746.0 +calvin zipper 65669.0 +calvin zipper 65739.0 +david allen 65588.0 +david allen 65617.0 +david brown 65637.0 +david brown 65760.0 +david davidson 65559.0 +david davidson 65756.0 +david davidson 65778.0 +david davidson 65779.0 +david ellison 65634.0 +david ellison 65724.0 +david ellison 65724.0 +david hernandez 65763.0 +david ichabod 65699.0 +david ichabod 65715.0 +david laertes 65762.0 +david nixon 65536.0 +david ovid 65623.0 +david ovid 65628.0 +david quirinius 65697.0 +david quirinius 65759.0 +david quirinius 65779.0 +david robinson 65762.0 +david robinson 65775.0 +david thompson 65550.0 +david underhill 65602.0 +david underhill 65662.0 +david underhill 65751.0 +david van buren 65625.0 +david van buren 65634.0 +david white 65678.0 +david xylophone 65537.0 +david xylophone 65670.0 +david xylophone 65764.0 +david young 65551.0 +david young 65694.0 +ethan allen 65747.0 +ethan brown 65539.0 +ethan brown 65617.0 +ethan brown 65685.0 +ethan brown 65685.0 +ethan brown 65722.0 +ethan brown 65733.0 +ethan carson 65742.0 +ethan ellison 65714.0 +ethan ellison 65732.0 +ethan falkner 65577.0 +ethan falkner 65614.0 +ethan garcia 65736.0 +ethan hernandez 65630.5 +ethan johnson 65536.0 +ethan king 65614.0 +ethan laertes 65562.0 +ethan laertes 65597.0 +ethan laertes 65628.0 +ethan laertes 65643.0 +ethan laertes 65680.0 +ethan laertes 65745.0 +ethan laertes 65760.0 +ethan miller 65712.0 +ethan nixon 65766.0 +ethan ovid 65697.0 +ethan polk 65589.0 +ethan polk 65615.0 +ethan polk 65622.0 +ethan polk 65622.0 +ethan quirinius 65542.0 +ethan quirinius 65591.0 +ethan quirinius 65706.0 +ethan robinson 65547.0 +ethan robinson 65659.0 +ethan underhill 65570.0 +ethan van buren 65572.0 +ethan white 65677.0 +ethan white 65788.0 +ethan xylophone 65595.0 +ethan zipper 65593.0 +ethan zipper 65680.0 +fred davidson 65552.0 +fred davidson 65595.0 +fred davidson 65721.0 +fred ellison 65548.0 +fred ellison 65691.0 +fred ellison 65771.0 +fred falkner 65637.0 +fred falkner 65648.0 +fred falkner 65783.0 +fred hernandez 65541.0 +fred ichabod 65572.0 +fred ichabod 65789.0 +fred johnson 65758.0 +fred king 65694.0 +fred king 65745.0 +fred laertes 65769.0 +fred miller 65536.0 +fred nixon 65560.0 +fred nixon 65612.0 +fred nixon 65703.0 +fred nixon 65705.0 +fred polk 65603.0 +fred polk 65656.0 +fred polk 65701.0 +fred polk 65706.0 +fred quirinius 65697.0 +fred quirinius 65782.0 +fred robinson 65623.0 +fred steinbeck 65544.0 +fred steinbeck 65651.0 +fred steinbeck 65755.0 +fred underhill 65629.0 +fred van buren 65537.0 +fred van buren 65561.0 +fred van buren 65745.0 +fred van buren 65789.0 +fred white 65589.0 +fred young 65594.0 +fred young 65773.0 +fred zipper 65553.0 +gabriella allen 65646.0 +gabriella allen 65677.0 +gabriella brown 65704.0 +gabriella brown 65753.0 +gabriella carson 65586.0 +gabriella davidson 65565.0 +gabriella ellison 65706.0 +gabriella ellison 65716.0 +gabriella falkner 65623.0 +gabriella falkner 65711.0 +gabriella falkner 65767.0 +gabriella garcia 65571.0 +gabriella hernandez 65587.0 +gabriella hernandez 65717.0 +gabriella ichabod 65559.0 +gabriella ichabod 65633.0 +gabriella ichabod 65702.0 +gabriella ichabod 65712.0 +gabriella ichabod 65717.0 +gabriella king 65651.0 +gabriella king 65657.0 +gabriella laertes 65781.0 +gabriella miller 65646.0 +gabriella ovid 65556.0 +gabriella ovid 65583.0 +gabriella polk 65701.0 +gabriella polk 65790.0 +gabriella steinbeck 65582.0 +gabriella steinbeck 65653.0 +gabriella thompson 65682.0 +gabriella thompson 65755.0 +gabriella thompson 65766.0 +gabriella van buren 65581.0 +gabriella van buren 65644.0 +gabriella white 65638.0 +gabriella young 65699.0 +gabriella young 65774.0 +gabriella zipper 65540.0 +gabriella zipper 65754.0 +holly allen 65596.0 +holly brown 65599.0 +holly brown 65619.0 +holly falkner 65720.0 +holly hernandez 65602.0 +holly hernandez 65686.0 +holly hernandez 65750.0 +holly hernandez 65788.0 +holly ichabod 65711.0 +holly ichabod 65749.0 +holly ichabod 65752.0 +holly johnson 65655.0 +holly johnson 65662.0 +holly johnson 65755.0 +holly king 65549.0 +holly king 65648.0 +holly laertes 65664.0 +holly miller 65653.0 +holly nixon 65539.0 +holly nixon 65658.0 +holly polk 65743.0 +holly polk 65751.0 +holly robinson 65564.0 +holly thompson 65538.0 +holly thompson 65578.0 +holly thompson 65713.0 +holly underhill 65634.0 +holly underhill 65654.0 +holly underhill 65721.0 +holly underhill 65759.0 +holly van buren 65727.0 +holly white 65536.0 +holly white 65602.0 +holly xylophone 65544.0 +holly young 65606.0 +holly young 65765.0 +holly zipper 65607.0 +holly zipper 65755.0 +irene allen 65556.0 +irene brown 65633.0 +irene brown 65664.0 +irene brown 65765.0 +irene carson 65590.0 +irene ellison 65659.0 +irene ellison 65696.0 +irene falkner 65620.0 +irene falkner 65661.0 +irene garcia 65660.0 +irene garcia 65711.0 +irene garcia 65787.0 +irene ichabod 65645.0 +irene ichabod 65722.0 +irene johnson 65583.0 +irene laertes 65664.0 +irene laertes 65710.0 +irene laertes 65722.0 +irene miller 65730.0 +irene nixon 65631.0 +irene nixon 65643.0 +irene nixon 65653.0 +irene ovid 65691.0 +irene ovid 65734.0 +irene ovid 65753.0 +irene polk 65551.0 +irene polk 65575.0 +irene polk 65579.0 +irene polk 65595.0 +irene polk 65610.0 +irene quirinius 65724.0 +irene quirinius 65769.0 +irene quirinius 65773.0 +irene robinson 65554.0 +irene steinbeck 65683.0 +irene thompson 65688.0 +irene underhill 65591.0 +irene underhill 65707.5 +irene van buren 65579.0 +irene van buren 65589.0 +irene xylophone 65775.0 +jessica brown 65588.0 +jessica carson 65553.0 +jessica carson 65672.0 +jessica carson 65747.0 +jessica davidson 65549.0 +jessica davidson 65606.0 +jessica davidson 65675.0 +jessica davidson 65727.0 +jessica ellison 65567.0 +jessica ellison 65663.0 +jessica falkner 65584.0 +jessica garcia 65676.0 +jessica garcia 65789.0 +jessica ichabod 65704.0 +jessica johnson 65607.0 +jessica johnson 65720.0 +jessica miller 65733.0 +jessica nixon 65590.0 +jessica nixon 65774.0 +jessica ovid 65582.0 +jessica ovid 65751.0 +jessica polk 65637.0 +jessica quirinius 65562.0 +jessica quirinius 65608.0 +jessica quirinius 65712.0 +jessica quirinius 65716.0 +jessica robinson 65576.0 +jessica thompson 65581.0 +jessica thompson 65675.0 +jessica underhill 65656.0 +jessica underhill 65702.0 +jessica underhill 65783.0 +jessica van buren 65615.0 +jessica white 65544.0 +jessica white 65570.0 +jessica white 65594.0 +jessica white 65673.0 +jessica white 65779.0 +jessica xylophone 65562.0 +jessica young 65623.0 +jessica young 65711.0 +jessica zipper 65600.0 +jessica zipper 65657.0 +jessica zipper 65778.0 +katie allen 65542.0 +katie brown 65590.0 +katie davidson 65619.0 +katie ellison 65675.0 +katie ellison 65699.0 +katie falkner 65728.0 +katie garcia 65625.0 +katie garcia 65747.0 +katie hernandez 65550.0 +katie ichabod 65658.0 +katie ichabod 65726.0 +katie ichabod 65757.0 +katie king 65629.0 +katie king 65647.0 +katie king 65776.0 +katie miller 65541.0 +katie miller 65661.0 +katie nixon 65669.0 +katie ovid 65681.0 +katie polk 65746.0 +katie polk 65784.0 +katie robinson 65697.0 +katie van buren 65643.0 +katie van buren 65730.0 +katie white 65620.0 +katie white 65719.0 +katie xylophone 65585.0 +katie young 65644.0 +katie young 65746.0 +katie young 65764.0 +katie zipper 65568.0 +katie zipper 65733.0 +luke allen 65547.0 +luke allen 65552.0 +luke allen 65576.0 +luke allen 65681.0 +luke allen 65776.0 +luke brown 65719.0 +luke davidson 65656.0 +luke davidson 65791.0 +luke ellison 65582.0 +luke ellison 65664.0 +luke ellison 65779.0 +luke falkner 65589.0 +luke falkner 65618.0 +luke garcia 65687.0 +luke garcia 65778.0 +luke ichabod 65629.0 +luke ichabod 65654.0 +luke johnson 65545.0 +luke johnson 65716.0 +luke johnson 65718.0 +luke laertes 65608.0 +luke laertes 65657.0 +luke laertes 65685.0 +luke laertes 65730.0 +luke laertes 65756.0 +luke miller 65752.0 +luke ovid 65569.0 +luke ovid 65693.0 +luke polk 65645.0 +luke polk 65658.0 +luke quirinius 65655.0 +luke robinson 65634.0 +luke robinson 65772.0 +luke thompson 65626.0 +luke underhill 65553.0 +luke underhill 65571.0 +luke underhill 65651.0 +luke van buren 65678.0 +luke white 65693.0 +luke xylophone 65597.0 +luke zipper 65641.0 +mike allen 65706.0 +mike brown 65654.0 +mike carson 65698.0 +mike carson 65700.0 +mike carson 65751.0 +mike davidson 65658.0 +mike davidson 65759.0 +mike ellison 65598.0 +mike ellison 65606.0 +mike ellison 65718.0 +mike ellison 65738.0 +mike ellison 65760.0 +mike falkner 65609.0 +mike garcia 65571.0 +mike garcia 65600.0 +mike garcia 65770.0 +mike hernandez 65548.0 +mike hernandez 65672.0 +mike ichabod 65621.0 +mike king 65563.0 +mike king 65586.0 +mike king 65591.0 +mike king 65642.0 +mike king 65769.0 +mike king 65776.0 +mike miller 65549.0 +mike nixon 65619.0 +mike nixon 65704.0 +mike polk 65619.0 +mike polk 65658.0 +mike polk 65704.0 +mike quirinius 65717.0 +mike steinbeck 65550.0 +mike steinbeck 65564.0 +mike steinbeck 65573.0 +mike steinbeck 65749.0 +mike van buren 65620.0 +mike van buren 65770.0 +mike white 65648.0 +mike white 65685.0 +mike white 65769.0 +mike white 65778.0 +mike young 65545.0 +mike young 65581.0 +mike young 65736.0 +mike zipper 65552.0 +mike zipper 65695.0 +mike zipper 65779.0 +nick allen 65641.0 +nick allen 65786.0 +nick brown 65724.0 +nick davidson 65601.0 +nick ellison 65691.0 +nick ellison 65745.0 +nick falkner 65583.0 +nick falkner 65676.0 +nick garcia 65712.0 +nick garcia 65720.0 +nick garcia 65723.0 +nick ichabod 65572.0 +nick ichabod 65681.0 +nick ichabod 65737.0 +nick johnson 65585.0 +nick johnson 65784.0 +nick laertes 65624.0 +nick miller 65757.0 +nick nixon 65650.0 +nick ovid 65719.0 +nick polk 65716.0 +nick quirinius 65588.0 +nick quirinius 65723.0 +nick robinson 65547.0 +nick robinson 65675.0 +nick steinbeck 65689.0 +nick thompson 65610.0 +nick underhill 65619.0 +nick van buren 65603.0 +nick xylophone 65644.0 +nick young 65654.0 +nick young 65660.0 +nick zipper 65757.0 +nick zipper 65765.0 +oscar allen 65644.0 +oscar brown 65614.0 +oscar carson 65537.0 +oscar carson 65548.0 +oscar carson 65549.0 +oscar carson 65624.0 +oscar carson 65697.0 +oscar davidson 65556.0 +oscar ellison 65630.0 +oscar ellison 65630.0 +oscar falkner 65692.0 +oscar garcia 65751.0 +oscar hernandez 65683.0 +oscar hernandez 65707.0 +oscar ichabod 65536.0 +oscar ichabod 65562.0 +oscar ichabod 65637.0 +oscar ichabod 65763.0 +oscar johnson 65645.0 +oscar johnson 65778.0 +oscar king 65541.0 +oscar king 65550.0 +oscar king 65787.0 +oscar laertes 65625.0 +oscar laertes 65690.0 +oscar laertes 65756.0 +oscar laertes 65790.0 +oscar nixon 65596.0 +oscar ovid 65536.0 +oscar ovid 65615.0 +oscar ovid 65665.5 +oscar polk 65541.0 +oscar polk 65643.0 +oscar quirinius 65541.0 +oscar quirinius 65560.0 +oscar quirinius 65689.0 +oscar quirinius 65720.0 +oscar robinson 65537.0 +oscar robinson 65658.0 +oscar robinson 65687.0 +oscar robinson 65782.0 +oscar steinbeck 65709.0 +oscar thompson 65542.0 +oscar thompson 65681.0 +oscar thompson 65727.0 +oscar thompson 65738.0 +oscar underhill 65626.0 +oscar van buren 65581.0 +oscar van buren 65635.0 +oscar van buren 65705.0 +oscar white 65552.0 +oscar white 65564.0 +oscar white 65671.0 +oscar white 65735.0 +oscar xylophone 65773.0 +oscar xylophone 65773.0 +oscar xylophone 65775.0 +oscar zipper 65568.0 +oscar zipper 65740.0 +oscar zipper 65777.0 +priscilla brown 65670.0 +priscilla brown 65690.0 +priscilla brown 65749.0 +priscilla carson 65658.0 +priscilla carson 65687.0 +priscilla carson 65755.0 +priscilla ichabod 65627.0 +priscilla ichabod 65759.0 +priscilla johnson 65543.0 +priscilla johnson 65668.0 +priscilla johnson 65674.5 +priscilla johnson 65681.0 +priscilla johnson 65755.0 +priscilla king 65646.0 +priscilla nixon 65564.0 +priscilla nixon 65600.0 +priscilla ovid 65541.0 +priscilla ovid 65790.0 +priscilla polk 65747.0 +priscilla quirinius 65672.0 +priscilla thompson 65654.0 +priscilla underhill 65715.0 +priscilla underhill 65729.0 +priscilla van buren 65607.0 +priscilla van buren 65685.0 +priscilla van buren 65749.0 +priscilla white 65652.0 +priscilla xylophone 65538.0 +priscilla xylophone 65763.0 +priscilla xylophone 65774.0 +priscilla young 65585.0 +priscilla young 65658.0 +priscilla zipper 65622.0 +priscilla zipper 65726.0 +quinn allen 65657.0 +quinn allen 65708.0 +quinn brown 65691.0 +quinn brown 65700.0 +quinn brown 65733.0 +quinn davidson 65549.0 +quinn davidson 65714.0 +quinn davidson 65776.0 +quinn davidson 65779.0 +quinn ellison 65705.0 +quinn ellison 65778.0 +quinn garcia 65568.0 +quinn garcia 65604.0 +quinn garcia 65610.0 +quinn garcia 65773.0 +quinn ichabod 65609.0 +quinn king 65558.0 +quinn king 65649.0 +quinn laertes 65542.0 +quinn laertes 65560.0 +quinn laertes 65627.0 +quinn nixon 65659.0 +quinn ovid 65699.0 +quinn quirinius 65747.0 +quinn robinson 65627.0 +quinn steinbeck 65578.0 +quinn steinbeck 65763.0 +quinn thompson 65643.0 +quinn thompson 65774.0 +quinn underhill 65549.0 +quinn underhill 65694.0 +quinn underhill 65767.0 +quinn van buren 65725.0 +quinn young 65647.0 +quinn zipper 65579.0 +quinn zipper 65693.0 +rachel allen 65661.0 +rachel allen 65709.0 +rachel brown 65586.0 +rachel brown 65587.0 +rachel brown 65587.0 +rachel brown 65610.0 +rachel brown 65693.0 +rachel carson 65677.0 +rachel carson 65682.0 +rachel davidson 65755.0 +rachel ellison 65761.0 +rachel falkner 65616.0 +rachel falkner 65681.0 +rachel falkner 65693.0 +rachel falkner 65764.0 +rachel johnson 65658.0 +rachel king 65604.0 +rachel king 65643.0 +rachel laertes 65562.0 +rachel laertes 65624.0 +rachel ovid 65721.0 +rachel ovid 65736.0 +rachel polk 65686.0 +rachel quirinius 65787.0 +rachel robinson 65544.0 +rachel robinson 65717.0 +rachel robinson 65724.0 +rachel thompson 65648.0 +rachel thompson 65662.0 +rachel thompson 65733.0 +rachel underhill 65667.0 +rachel white 65615.0 +rachel white 65717.0 +rachel young 65727.0 +rachel zipper 65757.0 +rachel zipper 65785.0 +sarah carson 65616.0 +sarah carson 65693.0 +sarah carson 65694.0 +sarah ellison 65611.0 +sarah falkner 65606.0 +sarah falkner 65680.0 +sarah garcia 65563.0 +sarah garcia 65638.0 +sarah garcia 65661.0 +sarah ichabod 65667.0 +sarah ichabod 65671.0 +sarah johnson 65659.0 +sarah johnson 65716.0 +sarah johnson 65731.0 +sarah johnson 65751.0 +sarah king 65650.0 +sarah king 65699.0 +sarah miller 65557.0 +sarah ovid 65550.0 +sarah robinson 65677.0 +sarah robinson 65763.0 +sarah steinbeck 65721.0 +sarah white 65622.0 +sarah white 65747.0 +sarah xylophone 65678.0 +sarah young 65595.0 +sarah zipper 65550.0 +tom brown 65593.0 +tom brown 65675.0 +tom carson 65539.0 +tom carson 65624.0 +tom carson 65780.0 +tom davidson 65780.0 +tom ellison 65578.0 +tom ellison 65670.0 +tom ellison 65756.0 +tom falkner 65574.0 +tom falkner 65625.0 +tom hernandez 65575.0 +tom hernandez 65632.0 +tom ichabod 65588.0 +tom johnson 65536.0 +tom johnson 65789.0 +tom king 65576.0 +tom laertes 65617.0 +tom laertes 65701.0 +tom miller 65594.0 +tom miller 65603.0 +tom miller 65704.0 +tom nixon 65672.0 +tom ovid 65628.0 +tom polk 65652.0 +tom polk 65742.0 +tom quirinius 65563.0 +tom quirinius 65783.0 +tom robinson 65626.0 +tom robinson 65632.0 +tom robinson 65691.0 +tom robinson 65758.0 +tom steinbeck 65666.0 +tom van buren 65621.0 +tom van buren 65652.0 +tom van buren 65669.0 +tom white 65548.0 +tom young 65544.0 +tom young 65546.0 +tom zipper 65789.0 +ulysses brown 65735.0 +ulysses carson 65602.0 +ulysses carson 65643.0 +ulysses carson 65703.0 +ulysses carson 65716.0 +ulysses davidson 65750.0 +ulysses ellison 65575.0 +ulysses garcia 65666.0 +ulysses hernandez 65651.0 +ulysses hernandez 65702.0 +ulysses hernandez 65786.0 +ulysses ichabod 65551.0 +ulysses ichabod 65566.0 +ulysses johnson 65776.0 +ulysses king 65649.0 +ulysses laertes 65691.0 +ulysses laertes 65711.0 +ulysses laertes 65781.0 +ulysses miller 65610.0 +ulysses miller 65637.0 +ulysses nixon 65603.0 +ulysses ovid 65656.0 +ulysses polk 65563.0 +ulysses polk 65580.0 +ulysses polk 65612.0 +ulysses polk 65777.0 +ulysses quirinius 65786.0 +ulysses robinson 65744.0 +ulysses steinbeck 65611.0 +ulysses steinbeck 65680.0 +ulysses thompson 65788.0 +ulysses underhill 65570.0 +ulysses underhill 65616.0 +ulysses underhill 65620.0 +ulysses underhill 65623.0 +ulysses underhill 65641.0 +ulysses underhill 65713.0 +ulysses underhill 65785.0 +ulysses van buren 65684.0 +ulysses white 65654.0 +ulysses white 65675.0 +ulysses xylophone 65623.0 +ulysses xylophone 65636.0 +ulysses xylophone 65781.0 +ulysses young 65675.0 +ulysses young 65736.0 +ulysses young 65748.0 +victor allen 65684.0 +victor allen 65707.0 +victor brown 65550.0 +victor brown 65555.0 +victor brown 65622.0 +victor brown 65673.0 +victor davidson 65579.0 +victor davidson 65628.0 +victor davidson 65783.0 +victor ellison 65641.0 +victor ellison 65782.0 +victor hernandez 65571.0 +victor hernandez 65659.0 +victor hernandez 65708.0 +victor hernandez 65735.0 +victor hernandez 65775.0 +victor johnson 65606.0 +victor johnson 65607.0 +victor johnson 65607.0 +victor king 65721.0 +victor king 65743.0 +victor laertes 65638.0 +victor laertes 65644.0 +victor miller 65570.0 +victor nixon 65709.0 +victor nixon 65791.0 +victor ovid 65649.0 +victor polk 65625.0 +victor quirinius 65620.0 +victor quirinius 65651.0 +victor robinson 65596.0 +victor robinson 65673.0 +victor steinbeck 65618.0 +victor steinbeck 65661.0 +victor steinbeck 65686.0 +victor thompson 65548.0 +victor van buren 65664.0 +victor van buren 65774.0 +victor white 65548.0 +victor white 65601.0 +victor xylophone 65549.0 +victor xylophone 65618.0 +victor xylophone 65644.0 +victor xylophone 65677.0 +victor xylophone 65755.0 +victor young 65628.0 +victor zipper 65743.0 +wendy allen 65628.0 +wendy allen 65711.0 +wendy allen 65782.0 +wendy brown 65580.0 +wendy brown 65657.0 +wendy ellison 65545.0 +wendy ellison 65603.0 +wendy falkner 65595.0 +wendy falkner 65604.0 +wendy falkner 65635.0 +wendy garcia 65659.0 +wendy garcia 65746.0 +wendy garcia 65747.0 +wendy garcia 65777.0 +wendy hernandez 65650.0 +wendy ichabod 65730.0 +wendy king 65586.0 +wendy king 65664.0 +wendy king 65670.0 +wendy laertes 65566.0 +wendy laertes 65683.0 +wendy laertes 65727.0 +wendy miller 65582.0 +wendy miller 65626.0 +wendy nixon 65611.0 +wendy nixon 65746.0 +wendy ovid 65589.0 +wendy ovid 65643.0 +wendy polk 65656.0 +wendy polk 65692.0 +wendy quirinius 65766.0 +wendy quirinius 65767.0 +wendy robinson 65622.0 +wendy robinson 65715.0 +wendy robinson 65774.0 +wendy steinbeck 65612.0 +wendy thompson 65650.0 +wendy thompson 65737.0 +wendy underhill 65662.0 +wendy underhill 65758.0 +wendy underhill 65775.0 +wendy van buren 65680.0 +wendy van buren 65699.0 +wendy white 65705.0 +wendy xylophone 65687.0 +wendy xylophone 65773.0 +wendy young 65674.0 +wendy young 65685.0 +xavier allen 65611.0 +xavier allen 65618.0 +xavier allen 65771.0 +xavier brown 65600.0 +xavier brown 65704.0 +xavier brown 65723.0 +xavier carson 65731.0 +xavier carson 65758.0 +xavier davidson 65644.0 +xavier davidson 65664.0 +xavier davidson 65755.0 +xavier ellison 65541.0 +xavier ellison 65622.0 +xavier garcia 65672.0 +xavier hernandez 65541.0 +xavier hernandez 65544.0 +xavier hernandez 65766.0 +xavier ichabod 65597.0 +xavier ichabod 65663.0 +xavier johnson 65655.0 +xavier johnson 65744.0 +xavier king 65590.0 +xavier king 65601.0 +xavier laertes 65743.0 +xavier ovid 65788.0 +xavier polk 65587.0 +xavier polk 65653.0 +xavier polk 65675.0 +xavier polk 65696.0 +xavier quirinius 65599.0 +xavier quirinius 65650.0 +xavier quirinius 65656.0 +xavier quirinius 65737.0 +xavier thompson 65608.0 +xavier underhill 65710.0 +xavier white 65703.0 +xavier white 65732.0 +xavier xylophone 65572.0 +xavier zipper 65561.0 +yuri allen 65565.0 +yuri allen 65682.0 +yuri brown 65538.0 +yuri brown 65688.0 +yuri carson 65670.0 +yuri carson 65769.0 +yuri ellison 65570.0 +yuri ellison 65581.0 +yuri falkner 65658.0 +yuri falkner 65681.0 +yuri garcia 65639.0 +yuri hernandez 65706.0 +yuri johnson 65587.0 +yuri johnson 65697.0 +yuri johnson 65712.0 +yuri king 65721.0 +yuri laertes 65637.0 +yuri laertes 65773.0 +yuri nixon 65635.0 +yuri nixon 65740.0 +yuri polk 65607.0 +yuri polk 65713.0 +yuri polk 65742.0 +yuri quirinius 65544.0 +yuri quirinius 65617.0 +yuri quirinius 65695.0 +yuri steinbeck 65592.0 +yuri steinbeck 65679.0 +yuri thompson 65676.0 +yuri underhill 65718.0 +yuri underhill 65750.0 +yuri white 65659.0 +yuri xylophone 65714.0 +zach allen 65667.0 +zach brown 65559.0 +zach brown 65588.0 +zach brown 65691.0 +zach brown 65759.0 +zach brown 65762.0 +zach carson 65572.0 +zach ellison 65748.0 +zach falkner 65620.0 +zach falkner 65627.0 +zach garcia 65544.0 +zach garcia 65623.0 +zach garcia 65629.0 +zach garcia 65764.5 +zach ichabod 65599.0 +zach ichabod 65612.0 +zach king 65556.0 +zach king 65702.0 +zach king 65773.0 +zach miller 65583.0 +zach miller 65665.0 +zach miller 65719.0 +zach ovid 65578.0 +zach ovid 65669.0 +zach ovid 65703.0 +zach ovid 65784.0 +zach quirinius 65691.0 +zach robinson 65599.0 +zach steinbeck 65602.0 +zach steinbeck 65695.0 +zach thompson 65636.0 +zach thompson 65696.0 +zach underhill 65573.0 +zach white 65733.0 +zach xylophone 65542.0 +zach xylophone 65780.0 +zach young 65576.0 +zach zipper 65579.0 +zach zipper 65649.0 +zach zipper 65676.0 diff --git a/sql/hive/src/test/resources/golden/windowing_udaf.q (deterministic)-3-d90b27fca067b0b3c48d873b3ef32af7 b/sql/hive/src/test/resources/golden/windowing_udaf.q (deterministic)-3-d90b27fca067b0b3c48d873b3ef32af7 new file mode 100644 index 0000000000000..072a8a891a836 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_udaf.q (deterministic)-3-d90b27fca067b0b3c48d873b3ef32af7 @@ -0,0 +1,1049 @@ +65536 +65536 +65536 +65536 +65536 +65536 +65537 +65537 +65537 +65537 +65538 +65538 +65538 +65539 +65539 +65539 +65540 +65541 +65541 +65541 +65541 +65541 +65541 +65541 +65541 +65542 +65542 +65542 +65542 +65543 +65543 +65544 +65544 +65544 +65544 +65544 +65544 +65544 +65544 +65545 +65545 +65545 +65547 +65547 +65547 +65547 +65548 +65548 +65548 +65548 +65548 +65548 +65548 +65549 +65549 +65549 +65549 +65549 +65549 +65549 +65549 +65550 +65550 +65550 +65550 +65550 +65550 +65550 +65551 +65551 +65551 +65552 +65552 +65552 +65552 +65552 +65553 +65553 +65553 +65553 +65554 +65554 +65555 +65556 +65556 +65556 +65557 +65557 +65558 +65559 +65559 +65559 +65559 +65560 +65560 +65560 +65560 +65561 +65562 +65562 +65562 +65562 +65562 +65562 +65563 +65563 +65563 +65563 +65563 +65564 +65564 +65564 +65564 +65564 +65564 +65564 +65564 +65565 +65565 +65566 +65566 +65567 +65568 +65568 +65568 +65569 +65570 +65570 +65570 +65570 +65570 +65571 +65571 +65571 +65572 +65572 +65572 +65572 +65572 +65573 +65573 +65573 +65574 +65574 +65574 +65575 +65575 +65575 +65575 +65576 +65576 +65576 +65576 +65577 +65578 +65578 +65578 +65578 +65578 +65578 +65579 +65579 +65579 +65579 +65579 +65580 +65580 +65580 +65581 +65581 +65581 +65581 +65581 +65582 +65582 +65582 +65582 +65582 +65583 +65583 +65583 +65583 +65584 +65584 +65584 +65584 +65585 +65585 +65585 +65585 +65585 +65586 +65586 +65586 +65586 +65586 +65587 +65587 +65587 +65587 +65588 +65588 +65588 +65588 +65588 +65589 +65589 +65589 +65589 +65589 +65590 +65590 +65590 +65590 +65590 +65591 +65591 +65591 +65591 +65592 +65593 +65593 +65594 +65594 +65594 +65594 +65595 +65595 +65595 +65595 +65595 +65595 +65596 +65596 +65596 +65596 +65596 +65597 +65597 +65597 +65598 +65599 +65599 +65599 +65600 +65600 +65600 +65600 +65601 +65601 +65601 +65602 +65602 +65602 +65602 +65603 +65603 +65603 +65603 +65603 +65604 +65604 +65604 +65605 +65606 +65606 +65606 +65606 +65606 +65607 +65607 +65607 +65607 +65607 +65607 +65608 +65608 +65608 +65608 +65609 +65610 +65610 +65610 +65610 +65610 +65610 +65611 +65611 +65611 +65612 +65612 +65612 +65614 +65614 +65614 +65615 +65615 +65615 +65616 +65616 +65617 +65617 +65617 +65617 +65618 +65618 +65618 +65618 +65618 +65619 +65619 +65619 +65619 +65619 +65619 +65620 +65620 +65620 +65620 +65620 +65620 +65621 +65621 +65622 +65622 +65622 +65622 +65622 +65622 +65623 +65623 +65623 +65623 +65623 +65623 +65623 +65624 +65624 +65624 +65624 +65625 +65625 +65625 +65625 +65626 +65626 +65626 +65626 +65627 +65627 +65627 +65627 +65628 +65628 +65628 +65628 +65628 +65628 +65629 +65629 +65629 +65630 +65630 +65631 +65632 +65632 +65632 +65633 +65633 +65633 +65633 +65634 +65634 +65634 +65634 +65635 +65635 +65635 +65636 +65636 +65636 +65636 +65637 +65637 +65637 +65637 +65637 +65637 +65637 +65638 +65638 +65638 +65639 +65640 +65641 +65641 +65641 +65641 +65642 +65643 +65643 +65643 +65643 +65643 +65643 +65643 +65643 +65643 +65643 +65644 +65644 +65644 +65644 +65644 +65645 +65645 +65645 +65646 +65646 +65646 +65647 +65647 +65648 +65648 +65648 +65648 +65649 +65649 +65649 +65650 +65650 +65650 +65650 +65650 +65650 +65650 +65651 +65651 +65651 +65651 +65651 +65651 +65652 +65652 +65652 +65653 +65653 +65653 +65653 +65654 +65654 +65654 +65654 +65654 +65654 +65654 +65654 +65654 +65655 +65655 +65656 +65656 +65656 +65656 +65656 +65656 +65657 +65657 +65657 +65657 +65657 +65658 +65658 +65658 +65658 +65658 +65658 +65658 +65658 +65658 +65658 +65658 +65659 +65659 +65659 +65659 +65659 +65659 +65659 +65659 +65660 +65660 +65660 +65661 +65661 +65661 +65661 +65661 +65662 +65662 +65662 +65662 +65662 +65662 +65663 +65663 +65663 +65663 +65664 +65664 +65664 +65664 +65664 +65664 +65664 +65665 +65666 +65666 +65667 +65667 +65667 +65667 +65667 +65667 +65668 +65669 +65669 +65669 +65669 +65669 +65669 +65670 +65670 +65670 +65670 +65670 +65671 +65671 +65671 +65672 +65672 +65672 +65672 +65672 +65672 +65673 +65673 +65673 +65673 +65673 +65674 +65674 +65674 +65674 +65675 +65675 +65675 +65675 +65675 +65675 +65675 +65675 +65676 +65676 +65676 +65677 +65677 +65677 +65677 +65677 +65677 +65678 +65678 +65678 +65678 +65679 +65679 +65680 +65680 +65680 +65680 +65680 +65680 +65680 +65681 +65681 +65681 +65681 +65681 +65682 +65682 +65682 +65683 +65683 +65683 +65683 +65684 +65684 +65684 +65684 +65685 +65685 +65685 +65685 +65685 +65685 +65686 +65686 +65686 +65687 +65687 +65687 +65687 +65688 +65688 +65689 +65689 +65690 +65690 +65691 +65691 +65691 +65691 +65691 +65691 +65691 +65691 +65692 +65692 +65693 +65693 +65693 +65693 +65693 +65693 +65694 +65694 +65694 +65695 +65695 +65695 +65695 +65695 +65696 +65696 +65696 +65696 +65697 +65697 +65697 +65697 +65697 +65697 +65698 +65698 +65698 +65699 +65699 +65699 +65699 +65699 +65699 +65700 +65700 +65700 +65701 +65701 +65701 +65702 +65702 +65702 +65702 +65702 +65703 +65703 +65703 +65703 +65703 +65704 +65704 +65704 +65704 +65704 +65704 +65705 +65705 +65705 +65705 +65706 +65706 +65706 +65706 +65706 +65706 +65707 +65707 +65708 +65708 +65709 +65709 +65709 +65710 +65711 +65711 +65711 +65711 +65711 +65711 +65712 +65712 +65712 +65712 +65712 +65713 +65713 +65713 +65713 +65713 +65714 +65714 +65714 +65715 +65715 +65715 +65715 +65715 +65716 +65716 +65716 +65716 +65716 +65716 +65717 +65717 +65717 +65717 +65717 +65718 +65718 +65718 +65718 +65719 +65719 +65719 +65719 +65720 +65720 +65720 +65720 +65720 +65720 +65721 +65721 +65721 +65721 +65721 +65721 +65721 +65722 +65722 +65722 +65722 +65723 +65723 +65724 +65724 +65724 +65724 +65724 +65724 +65725 +65726 +65726 +65726 +65726 +65727 +65727 +65727 +65727 +65727 +65728 +65728 +65729 +65730 +65730 +65730 +65730 +65731 +65731 +65731 +65731 +65732 +65732 +65732 +65733 +65733 +65733 +65733 +65733 +65733 +65734 +65734 +65735 +65735 +65735 +65736 +65736 +65736 +65736 +65737 +65737 +65737 +65737 +65737 +65738 +65738 +65738 +65738 +65739 +65739 +65739 +65740 +65740 +65740 +65741 +65742 +65742 +65742 +65743 +65743 +65743 +65743 +65744 +65744 +65745 +65745 +65745 +65745 +65746 +65746 +65746 +65746 +65747 +65747 +65747 +65747 +65747 +65747 +65747 +65748 +65748 +65749 +65749 +65749 +65749 +65749 +65750 +65750 +65750 +65750 +65750 +65751 +65751 +65751 +65751 +65751 +65752 +65752 +65753 +65753 +65754 +65754 +65755 +65755 +65755 +65755 +65755 +65755 +65755 +65755 +65755 +65756 +65756 +65756 +65756 +65756 +65757 +65757 +65757 +65757 +65757 +65758 +65758 +65758 +65758 +65758 +65758 +65759 +65759 +65759 +65759 +65759 +65760 +65760 +65760 +65760 +65760 +65761 +65762 +65762 +65762 +65762 +65762 +65763 +65763 +65763 +65763 +65763 +65764 +65764 +65764 +65765 +65765 +65765 +65766 +65766 +65766 +65766 +65766 +65767 +65767 +65767 +65768 +65769 +65769 +65769 +65769 +65769 +65769 +65769 +65770 +65770 +65771 +65771 +65771 +65772 +65773 +65773 +65773 +65773 +65773 +65773 +65773 +65773 +65774 +65774 +65774 +65774 +65774 +65774 +65775 +65775 +65775 +65775 +65775 +65775 +65776 +65776 +65776 +65776 +65776 +65776 +65776 +65777 +65777 +65777 +65777 +65777 +65777 +65778 +65778 +65778 +65778 +65778 +65778 +65778 +65778 +65778 +65779 +65779 +65779 +65779 +65779 +65779 +65779 +65780 +65780 +65780 +65781 +65781 +65781 +65782 +65782 +65782 +65782 +65782 +65783 +65783 +65783 +65783 +65783 +65783 +65783 +65784 +65784 +65784 +65784 +65784 +65785 +65785 +65786 +65786 +65786 +65786 +65786 +65787 +65787 +65787 +65787 +65787 +65788 +65788 +65788 +65788 +65789 +65789 +65789 +65789 +65789 +65789 +65789 +65789 +65789 +65789 +65790 +65790 +65790 +65791 +65791 diff --git a/sql/hive/src/test/resources/golden/windowing_udaf.q (deterministic)-4-f2e4d659b65a833e9281b6786d3d55c1 b/sql/hive/src/test/resources/golden/windowing_udaf.q (deterministic)-4-f2e4d659b65a833e9281b6786d3d55c1 new file mode 100644 index 0000000000000..9cc7e7ea6c2b2 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_udaf.q (deterministic)-4-f2e4d659b65a833e9281b6786d3d55c1 @@ -0,0 +1,1049 @@ + 24.37875 + 27.900000000000002 + 43.64 +alice allen 16.919999999999998 +alice allen 20.39 +alice allen 23.59 +alice brown 6.91 +alice carson 41.74 +alice davidson 26.346000000000004 +alice falkner 32.166666666666664 +alice garcia 15.412 +alice hernandez 19.958181818181817 +alice hernandez 30.482857142857142 +alice johnson 25.51 +alice king 13.085 +alice king 25.616666666666664 +alice king 42.335 +alice laertes 20.549999999999997 +alice laertes 30.436 +alice miller 29.563333333333333 +alice nixon 19.28666666666667 +alice nixon 24.5625 +alice nixon 32.99 +alice ovid 31.35 +alice polk 17.863999999999997 +alice quirinius 19.032857142857143 +alice quirinius 23.9425 +alice robinson 23.338750000000005 +alice robinson 39.85 +alice steinbeck 22.862000000000002 +alice steinbeck 26.328000000000003 +alice steinbeck 27.08777777777778 +alice underhill 24.032222222222224 +alice van buren 19.642000000000003 +alice xylophone 24.438000000000002 +alice xylophone 28.739999999999995 +alice xylophone 30.0825 +alice zipper 26.3 +alice zipper 28.735000000000003 +alice zipper 31.05545454545455 +bob brown 12.902222222222223 +bob brown 13.945 +bob brown 33.843333333333334 +bob carson 28.627999999999997 +bob davidson 19.8525 +bob davidson 23.482 +bob davidson 24.67 +bob ellison 16.315714285714286 +bob ellison 18.4 +bob ellison 26.913999999999998 +bob ellison 27.59 +bob falkner 9.27 +bob garcia 11.63 +bob garcia 22.221249999999998 +bob garcia 23.59636363636364 +bob garcia 26.88857142857143 +bob garcia 28.715000000000003 +bob hernandez 37.23 +bob ichabod 28.33875 +bob king 8.615 +bob king 19.77 +bob king 26.7325 +bob laertes 21.33 +bob laertes 37.88 +bob miller 25.495 +bob ovid 25.675 +bob ovid 25.83 +bob ovid 28.37875 +bob ovid 32.5025 +bob polk 9.74 +bob quirinius 34.57 +bob steinbeck 9.725 +bob van buren 29.552857142857142 +bob white 17.685 +bob white 29.46285714285715 +bob xylophone 17.03 +bob xylophone 33.24 +bob young 19.824 +bob zipper 24.095 +bob zipper 33.36 +bob zipper 34.99 +calvin allen 21.3 +calvin brown 20.808 +calvin brown 24.16 +calvin brown 24.636666666666667 +calvin carson 22.815 +calvin davidson 22.116666666666664 +calvin davidson 22.364 +calvin ellison 24.92 +calvin falkner 18.343999999999998 +calvin falkner 19.56 +calvin falkner 22.946000000000005 +calvin falkner 23.327777777777776 +calvin falkner 23.974999999999998 +calvin falkner 33.382 +calvin garcia 17.285 +calvin hernandez 12.663333333333334 +calvin johnson 24.898571428571433 +calvin laertes 28.105 +calvin laertes 28.362000000000002 +calvin nixon 26.784285714285716 +calvin nixon 27.36 +calvin nixon 32.282 +calvin ovid 22.063333333333336 +calvin ovid 22.81500000000001 +calvin ovid 25.495714285714286 +calvin ovid 30.926666666666666 +calvin polk 27.820000000000004 +calvin quirinius 16.28 +calvin quirinius 25.552500000000002 +calvin robinson 31.814999999999998 +calvin steinbeck 12.85 +calvin steinbeck 14.939999999999998 +calvin steinbeck 17.535 +calvin thompson 28.592857142857145 +calvin thompson 40.79 +calvin underhill 24.062 +calvin van buren 26.525 +calvin van buren 28.865 +calvin white 28.256249999999998 +calvin white 43.275 +calvin xylophone 24.13111111111111 +calvin xylophone 25.27 +calvin xylophone 36.455 +calvin young 19.06 +calvin young 21.455999999999996 +calvin zipper 10.674999999999999 +calvin zipper 26.012857142857143 +david allen 25.134285714285713 +david allen 41.72333333333333 +david brown 8.52 +david brown 28.968181818181815 +david davidson 17.63 +david davidson 26.563333333333336 +david davidson 30.7325 +david davidson 33.33 +david ellison 23.79909090909091 +david ellison 24.74888888888889 +david ellison 26.198571428571427 +david hernandez 27.766 +david ichabod 16.66 +david ichabod 19.538 +david laertes 24.587500000000002 +david nixon 26.01375 +david ovid 24.131428571428575 +david ovid 32.72 +david quirinius 16.5 +david quirinius 25.08 +david quirinius 29.415 +david robinson 22.2175 +david robinson 30.99 +david thompson 25.38 +david underhill 1.17 +david underhill 21.546666666666667 +david underhill 28.26 +david van buren 26.45833333333334 +david van buren 35.7825 +david white 15.833333333333334 +david xylophone 10.71 +david xylophone 26.341428571428565 +david xylophone 33.224000000000004 +david young 9.64 +david young 21.22 +ethan allen 22.68 +ethan brown 19.37 +ethan brown 21.58666666666667 +ethan brown 21.799999999999997 +ethan brown 29.099999999999998 +ethan brown 32.43666666666667 +ethan brown 39.84 +ethan carson 24.15666666666667 +ethan ellison 27.80777777777778 +ethan ellison 48.71 +ethan falkner 17.993333333333336 +ethan falkner 26.775000000000002 +ethan garcia 19.15 +ethan hernandez 25.081111111111113 +ethan johnson 32.81875 +ethan king 19.51 +ethan laertes 16.463 +ethan laertes 17.625999999999998 +ethan laertes 25.020714285714288 +ethan laertes 26.697142857142858 +ethan laertes 28.14 +ethan laertes 29.668571428571425 +ethan laertes 36.589999999999996 +ethan miller 24.326666666666664 +ethan nixon 34.78666666666667 +ethan ovid 20.642857142857142 +ethan polk 6.98 +ethan polk 12.756666666666666 +ethan polk 30.324 +ethan polk 40.46 +ethan quirinius 23.419999999999998 +ethan quirinius 24.36 +ethan quirinius 29.068 +ethan robinson 24.463750000000005 +ethan robinson 31.630000000000003 +ethan underhill 19.86 +ethan van buren 22.241999999999997 +ethan white 31.3175 +ethan white 32.87 +ethan xylophone 30.996000000000002 +ethan zipper 22.728333333333335 +ethan zipper 29.66 +fred davidson 30.116666666666667 +fred davidson 33.55200000000001 +fred davidson 39.37 +fred ellison 16.72 +fred ellison 17.462 +fred ellison 35.1 +fred falkner 14.51 +fred falkner 27.207000000000004 +fred falkner 27.887500000000003 +fred hernandez 36.045 +fred ichabod 29.017000000000003 +fred ichabod 30.405000000000005 +fred johnson 16.9925 +fred king 20.024 +fred king 32.54666666666667 +fred laertes 25.610000000000003 +fred miller 25.92 +fred nixon 14.915 +fred nixon 21.830000000000002 +fred nixon 24.4125 +fred nixon 31.360000000000003 +fred polk 18.698 +fred polk 19.743000000000002 +fred polk 20.96 +fred polk 31.11 +fred quirinius 20.085 +fred quirinius 33.9 +fred robinson 22.502 +fred steinbeck 21.123749999999998 +fred steinbeck 25.572 +fred steinbeck 30.81 +fred underhill 29.198888888888884 +fred van buren 21.34 +fred van buren 23.285 +fred van buren 26.520000000000003 +fred van buren 33.6 +fred white 21.41 +fred young 16.876250000000002 +fred young 20.996666666666666 +fred zipper 23.627499999999998 +gabriella allen 24.113333333333333 +gabriella allen 28.4725 +gabriella brown 29.963333333333335 +gabriella brown 30.65222222222222 +gabriella carson 16.6325 +gabriella davidson 34.52 +gabriella ellison 20.18 +gabriella ellison 29.62 +gabriella falkner 14.37 +gabriella falkner 17.738333333333333 +gabriella falkner 28.61 +gabriella garcia 39.025 +gabriella hernandez 20.818333333333335 +gabriella hernandez 24.601666666666663 +gabriella ichabod 10.4925 +gabriella ichabod 20.686666666666667 +gabriella ichabod 23.185 +gabriella ichabod 23.43 +gabriella ichabod 27.44636363636364 +gabriella king 13.645 +gabriella king 22.23 +gabriella laertes 23.735 +gabriella miller 17.165 +gabriella ovid 22.884545454545453 +gabriella ovid 25.29 +gabriella polk 20.38714285714286 +gabriella polk 25.832000000000004 +gabriella steinbeck 6.226666666666667 +gabriella steinbeck 29.683333333333337 +gabriella thompson 25.565454545454546 +gabriella thompson 29.031 +gabriella thompson 29.122500000000006 +gabriella van buren 24.353 +gabriella van buren 34.21666666666667 +gabriella white 36.5175 +gabriella young 21.28142857142857 +gabriella young 21.32 +gabriella zipper 21.798461538461545 +gabriella zipper 28.676666666666666 +holly allen 27.18 +holly brown 22.76 +holly brown 30.950000000000003 +holly falkner 29.666666666666668 +holly hernandez 19.875 +holly hernandez 23.7075 +holly hernandez 24.5 +holly hernandez 26.50333333333333 +holly ichabod 23.262857142857143 +holly ichabod 25.85090909090909 +holly ichabod 29.521666666666665 +holly johnson 18.939999999999998 +holly johnson 23.2625 +holly johnson 26.49285714285714 +holly king 20.61333333333333 +holly king 30.95888888888889 +holly laertes 17.509999999999998 +holly miller 40.8975 +holly nixon 27.775714285714287 +holly nixon 30.642500000000002 +holly polk 21.02 +holly polk 24.446666666666665 +holly robinson 26.083750000000006 +holly thompson 18.801428571428573 +holly thompson 23.91 +holly thompson 29.97125 +holly underhill 18.19 +holly underhill 22.22888888888889 +holly underhill 22.813333333333336 +holly underhill 30.613999999999997 +holly van buren 20.113333333333333 +holly white 25.284999999999997 +holly white 41.0125 +holly xylophone 26.88571428571429 +holly young 30.8425 +holly young 33.24333333333334 +holly zipper 27.784000000000002 +holly zipper 28.384285714285713 +irene allen 34.605000000000004 +irene brown 18.740000000000002 +irene brown 28.974999999999998 +irene brown 32.230000000000004 +irene carson 25.665833333333335 +irene ellison 10.225000000000001 +irene ellison 26.119999999999997 +irene falkner 9.94 +irene falkner 19.41 +irene garcia 9.790000000000001 +irene garcia 19.666666666666668 +irene garcia 21.22666666666667 +irene ichabod 20.956666666666667 +irene ichabod 24.488333333333333 +irene johnson 25.34 +irene laertes 15.85 +irene laertes 21.573333333333334 +irene laertes 22.041999999999998 +irene miller 34.994285714285716 +irene nixon 22.52 +irene nixon 32.485 +irene nixon 33.165 +irene ovid 17.73 +irene ovid 22.96 +irene ovid 30.92 +irene polk 5.35 +irene polk 25.535 +irene polk 33.76 +irene polk 35.05 +irene polk 45.14 +irene quirinius 38.36 +irene quirinius 41.864999999999995 +irene quirinius 42.0 +irene robinson 30.86 +irene steinbeck 15.08 +irene thompson 28.419999999999998 +irene underhill 27.977999999999998 +irene underhill 28.438 +irene van buren 26.93625 +irene van buren 27.797999999999995 +irene xylophone 29.10454545454546 +jessica brown 38.325 +jessica carson 16.038 +jessica carson 29.668333333333337 +jessica carson 33.06 +jessica davidson 18.926 +jessica davidson 26.2975 +jessica davidson 27.611428571428572 +jessica davidson 29.86 +jessica ellison 26.873333333333335 +jessica ellison 27.123333333333335 +jessica falkner 21.75142857142858 +jessica garcia 16.939090909090908 +jessica garcia 26.48 +jessica ichabod 28.971666666666664 +jessica johnson 21.601428571428574 +jessica johnson 24.42 +jessica miller 26.90571428571429 +jessica nixon 19.15 +jessica nixon 27.025000000000002 +jessica ovid 30.72285714285714 +jessica ovid 30.895 +jessica polk 27.912857142857145 +jessica quirinius 17.05 +jessica quirinius 21.529999999999998 +jessica quirinius 25.16 +jessica quirinius 26.347999999999995 +jessica robinson 24.322857142857142 +jessica thompson 28.658000000000005 +jessica thompson 30.873636363636365 +jessica underhill 14.6725 +jessica underhill 25.831666666666667 +jessica underhill 31.345000000000002 +jessica van buren 19.575 +jessica white 18.35 +jessica white 19.175 +jessica white 20.812 +jessica white 26.0 +jessica white 29.307142857142857 +jessica xylophone 22.26 +jessica young 27.9525 +jessica young 37.61333333333334 +jessica zipper 7.03 +jessica zipper 15.794999999999998 +jessica zipper 19.95 +katie allen 27.283846153846152 +katie brown 24.156666666666666 +katie davidson 13.498000000000001 +katie ellison 19.2 +katie ellison 24.888571428571428 +katie falkner 28.959999999999997 +katie garcia 28.287142857142857 +katie garcia 36.196666666666665 +katie hernandez 25.14428571428572 +katie ichabod 19.363333333333333 +katie ichabod 20.458571428571428 +katie ichabod 28.924999999999997 +katie king 21.64125 +katie king 21.855 +katie king 22.895 +katie miller 16.263333333333335 +katie miller 30.274285714285718 +katie nixon 25.022499999999997 +katie ovid 24.055000000000003 +katie polk 21.296666666666667 +katie polk 32.03 +katie robinson 36.26 +katie van buren 28.332 +katie van buren 31.408000000000005 +katie white 23.48 +katie white 26.236666666666665 +katie xylophone 32.415 +katie young 18.209999999999997 +katie young 22.88125 +katie young 28.39888888888889 +katie zipper 10.285 +katie zipper 27.495 +luke allen 9.42 +luke allen 21.374615384615385 +luke allen 25.32 +luke allen 27.174999999999997 +luke allen 35.434 +luke brown 25.08 +luke davidson 28.205 +luke davidson 28.790000000000003 +luke ellison 7.8 +luke ellison 16.04 +luke ellison 23.426666666666666 +luke falkner 18.0 +luke falkner 22.19 +luke garcia 29.619999999999997 +luke garcia 32.722 +luke ichabod 21.150000000000002 +luke ichabod 32.78142857142857 +luke johnson 21.58666666666667 +luke johnson 23.03 +luke johnson 23.054 +luke laertes 20.264 +luke laertes 33.72 +luke laertes 39.8 +luke laertes 41.36 +luke laertes 42.254999999999995 +luke miller 20.054444444444446 +luke ovid 19.819999999999997 +luke ovid 30.832857142857147 +luke polk 24.348750000000003 +luke polk 26.57625 +luke quirinius 38.07 +luke robinson 30.119999999999994 +luke robinson 30.31375 +luke thompson 29.026874999999997 +luke underhill 21.735714285714288 +luke underhill 22.175 +luke underhill 26.785714285714285 +luke van buren 17.072222222222223 +luke white 29.063333333333333 +luke xylophone 28.994 +luke zipper 33.995 +mike allen 32.78 +mike brown 27.592222222222222 +mike carson 28.8675 +mike carson 29.88 +mike carson 32.07142857142857 +mike davidson 21.240000000000002 +mike davidson 46.31 +mike ellison 20.5275 +mike ellison 21.99 +mike ellison 24.36 +mike ellison 24.511111111111113 +mike ellison 27.703333333333337 +mike falkner 40.335 +mike garcia 24.3525 +mike garcia 24.582 +mike garcia 35.12 +mike hernandez 8.783333333333333 +mike hernandez 19.40666666666667 +mike ichabod 29.120000000000005 +mike king 14.256666666666668 +mike king 17.889999999999997 +mike king 20.493333333333336 +mike king 23.86 +mike king 26.081 +mike king 30.974 +mike miller 29.275 +mike nixon 17.306 +mike nixon 25.572 +mike polk 18.96 +mike polk 23.75142857142857 +mike polk 33.42 +mike quirinius 19.37375 +mike steinbeck 14.155 +mike steinbeck 19.305833333333332 +mike steinbeck 20.721249999999998 +mike steinbeck 31.75 +mike van buren 15.520000000000001 +mike van buren 25.828333333333333 +mike white 19.13111111111111 +mike white 22.4025 +mike white 24.7725 +mike white 35.235 +mike young 1.5 +mike young 24.679 +mike young 34.02833333333333 +mike zipper 17.97 +mike zipper 26.247333333333337 +mike zipper 44.169999999999995 +nick allen 23.744999999999997 +nick allen 36.93 +nick brown 27.669999999999998 +nick davidson 31.97285714285714 +nick ellison 23.061666666666667 +nick ellison 27.676666666666666 +nick falkner 22.555714285714284 +nick falkner 27.46 +nick garcia 17.465 +nick garcia 18.854 +nick garcia 33.60333333333333 +nick ichabod 19.231428571428573 +nick ichabod 27.645000000000003 +nick ichabod 35.836666666666666 +nick johnson 5.58 +nick johnson 25.274 +nick laertes 26.57857142857143 +nick miller 22.208333333333332 +nick nixon 16.107499999999998 +nick ovid 31.350000000000005 +nick polk 35.70333333333334 +nick quirinius 20.753333333333334 +nick quirinius 30.573333333333334 +nick robinson 21.48 +nick robinson 23.185 +nick steinbeck 19.56555555555556 +nick thompson 31.474999999999998 +nick underhill 38.24 +nick van buren 20.77375 +nick xylophone 30.909999999999997 +nick young 10.725000000000001 +nick young 24.95 +nick zipper 16.185000000000002 +nick zipper 34.72 +oscar allen 24.645 +oscar brown 39.55 +oscar carson 21.893333333333334 +oscar carson 22.868 +oscar carson 27.4875 +oscar carson 28.09428571428571 +oscar carson 30.373333333333335 +oscar davidson 9.046666666666667 +oscar ellison 24.185000000000002 +oscar ellison 30.1675 +oscar falkner 19.295 +oscar garcia 22.495833333333334 +oscar hernandez 16.6825 +oscar hernandez 25.736 +oscar ichabod 17.64 +oscar ichabod 21.11 +oscar ichabod 23.508000000000003 +oscar ichabod 30.392222222222227 +oscar johnson 19.9375 +oscar johnson 21.114444444444445 +oscar king 24.590000000000003 +oscar king 26.675 +oscar king 39.6 +oscar laertes 14.975 +oscar laertes 15.525 +oscar laertes 22.6 +oscar laertes 41.6 +oscar nixon 25.4025 +oscar ovid 24.854285714285712 +oscar ovid 25.309 +oscar ovid 29.63 +oscar polk 21.235999999999997 +oscar polk 21.27 +oscar quirinius 24.200000000000003 +oscar quirinius 24.391428571428573 +oscar quirinius 27.83285714285714 +oscar quirinius 27.853333333333328 +oscar robinson 12.3625 +oscar robinson 12.545 +oscar robinson 20.234 +oscar robinson 28.071666666666673 +oscar steinbeck 31.101111111111113 +oscar thompson 19.4875 +oscar thompson 19.975714285714286 +oscar thompson 21.1425 +oscar thompson 21.166363636363638 +oscar underhill 27.644 +oscar van buren 25.843333333333334 +oscar van buren 29.073333333333334 +oscar van buren 29.682727272727274 +oscar white 19.0775 +oscar white 23.483333333333334 +oscar white 24.705000000000002 +oscar white 28.0075 +oscar xylophone 30.020000000000003 +oscar xylophone 30.46833333333333 +oscar xylophone 33.64 +oscar zipper 21.69 +oscar zipper 23.478 +oscar zipper 31.36 +priscilla brown 14.222 +priscilla brown 27.044999999999998 +priscilla brown 31.14769230769231 +priscilla carson 14.33 +priscilla carson 18.951428571428572 +priscilla carson 27.084999999999997 +priscilla ichabod 28.160999999999994 +priscilla ichabod 49.46 +priscilla johnson 8.365 +priscilla johnson 18.176666666666666 +priscilla johnson 25.02666666666667 +priscilla johnson 26.918333333333337 +priscilla johnson 30.695999999999998 +priscilla king 19.747142857142855 +priscilla nixon 29.035555555555554 +priscilla nixon 30.27333333333333 +priscilla ovid 13.591999999999999 +priscilla ovid 35.879999999999995 +priscilla polk 23.12 +priscilla quirinius 21.826666666666668 +priscilla thompson 20.44 +priscilla underhill 28.23 +priscilla underhill 34.33200000000001 +priscilla van buren 18.122857142857143 +priscilla van buren 20.16 +priscilla van buren 26.447999999999997 +priscilla white 26.37769230769231 +priscilla xylophone 13.95 +priscilla xylophone 20.596666666666668 +priscilla xylophone 27.22 +priscilla young 29.19 +priscilla young 46.28 +priscilla zipper 11.64 +priscilla zipper 31.159999999999997 +quinn allen 26.347272727272728 +quinn allen 26.85833333333333 +quinn brown 26.822857142857146 +quinn brown 30.406000000000006 +quinn brown 41.53 +quinn davidson 17.375714285714288 +quinn davidson 20.22666666666667 +quinn davidson 25.6375 +quinn davidson 30.173333333333332 +quinn ellison 23.052 +quinn ellison 40.565 +quinn garcia 20.544 +quinn garcia 24.104999999999997 +quinn garcia 25.174 +quinn garcia 28.446000000000005 +quinn ichabod 15.12 +quinn king 12.73 +quinn king 15.12125 +quinn laertes 17.29 +quinn laertes 28.221666666666668 +quinn laertes 32.96 +quinn nixon 26.034000000000002 +quinn ovid 28.71 +quinn quirinius 8.61 +quinn robinson 16.852 +quinn steinbeck 30.093333333333334 +quinn steinbeck 49.21 +quinn thompson 7.365 +quinn thompson 33.43125 +quinn underhill 24.045 +quinn underhill 27.905454545454543 +quinn underhill 31.21 +quinn van buren 27.807692307692314 +quinn young 30.56 +quinn zipper 18.31 +quinn zipper 21.380000000000003 +rachel allen 32.501666666666665 +rachel allen 46.57 +rachel brown 23.08 +rachel brown 23.880000000000003 +rachel brown 24.43 +rachel brown 34.11 +rachel brown 35.345 +rachel carson 27.468125 +rachel carson 37.446666666666665 +rachel davidson 22.75 +rachel ellison 22.848333333333333 +rachel falkner 18.78125 +rachel falkner 28.876250000000002 +rachel falkner 29.577777777777776 +rachel falkner 31.831249999999997 +rachel johnson 31.108000000000004 +rachel king 17.4175 +rachel king 30.873749999999998 +rachel laertes 17.470000000000002 +rachel laertes 33.51 +rachel ovid 3.03 +rachel ovid 15.38 +rachel polk 18.564285714285713 +rachel quirinius 31.692500000000003 +rachel robinson 0.6 +rachel robinson 23.953333333333333 +rachel robinson 37.645 +rachel thompson 11.96 +rachel thompson 29.484 +rachel thompson 38.43 +rachel underhill 27.55333333333333 +rachel white 23.511428571428574 +rachel white 33.7 +rachel young 24.85166666666667 +rachel zipper 22.85 +rachel zipper 37.382 +sarah carson 10.38 +sarah carson 22.639 +sarah carson 44.92 +sarah ellison 16.36 +sarah falkner 29.34875 +sarah falkner 29.64125 +sarah garcia 11.296666666666667 +sarah garcia 20.723333333333333 +sarah garcia 24.115 +sarah ichabod 26.948333333333327 +sarah ichabod 33.80428571428571 +sarah johnson 18.3925 +sarah johnson 23.087500000000002 +sarah johnson 26.57857142857143 +sarah johnson 37.01 +sarah king 9.556666666666667 +sarah king 25.6125 +sarah miller 19.14875 +sarah ovid 29.205 +sarah robinson 11.326666666666668 +sarah robinson 35.809999999999995 +sarah steinbeck 23.26 +sarah white 21.75111111111111 +sarah white 26.850000000000005 +sarah xylophone 33.40571428571429 +sarah young 30.66 +sarah zipper 29.521666666666672 +tom brown 16.38 +tom brown 23.645 +tom carson 23.630000000000003 +tom carson 31.935 +tom carson 41.83 +tom davidson 30.404285714285717 +tom ellison 27.056 +tom ellison 27.401999999999997 +tom ellison 29.812 +tom falkner 15.901999999999997 +tom falkner 25.49857142857143 +tom hernandez 11.418000000000001 +tom hernandez 30.705000000000002 +tom ichabod 14.83 +tom johnson 30.748571428571434 +tom johnson 37.086666666666666 +tom king 17.923333333333332 +tom laertes 19.201666666666668 +tom laertes 22.276666666666667 +tom miller 17.9925 +tom miller 19.791666666666668 +tom miller 19.9225 +tom nixon 25.70625 +tom ovid 29.66 +tom polk 27.0975 +tom polk 28.646666666666672 +tom quirinius 37.68333333333333 +tom quirinius 38.28 +tom robinson 18.07 +tom robinson 19.094 +tom robinson 27.34125 +tom robinson 31.135714285714283 +tom steinbeck 32.70333333333333 +tom van buren 20.723333333333333 +tom van buren 24.8525 +tom van buren 31.631666666666664 +tom white 25.646000000000004 +tom young 3.12 +tom young 19.588333333333335 +tom zipper 23.317272727272726 +ulysses brown 16.196666666666665 +ulysses carson 16.3475 +ulysses carson 22.448181818181823 +ulysses carson 28.258 +ulysses carson 32.10833333333333 +ulysses davidson 37.775 +ulysses ellison 30.517000000000003 +ulysses garcia 32.92 +ulysses hernandez 13.877500000000001 +ulysses hernandez 20.856666666666666 +ulysses hernandez 21.32625 +ulysses ichabod 3.29 +ulysses ichabod 24.629999999999995 +ulysses johnson 32.208333333333336 +ulysses king 25.29111111111111 +ulysses laertes 14.936666666666667 +ulysses laertes 25.89 +ulysses laertes 26.63 +ulysses miller 2.36 +ulysses miller 26.403333333333336 +ulysses nixon 34.4575 +ulysses ovid 23.810000000000002 +ulysses polk 22.4075 +ulysses polk 26.778000000000002 +ulysses polk 38.73166666666667 +ulysses polk 47.68 +ulysses quirinius 33.07833333333333 +ulysses robinson 17.386666666666667 +ulysses steinbeck 22.2675 +ulysses steinbeck 24.904000000000003 +ulysses thompson 22.687142857142856 +ulysses underhill 6.66 +ulysses underhill 22.539 +ulysses underhill 24.853333333333335 +ulysses underhill 27.314 +ulysses underhill 29.424999999999997 +ulysses underhill 32.905 +ulysses underhill 41.653333333333336 +ulysses van buren 21.868181818181817 +ulysses white 15.296666666666667 +ulysses white 28.343333333333334 +ulysses xylophone 24.718 +ulysses xylophone 30.205 +ulysses xylophone 35.61 +ulysses young 21.56 +ulysses young 32.28125 +ulysses young 37.275 +victor allen 23.548000000000002 +victor allen 24.759999999999998 +victor brown 22.10181818181818 +victor brown 23.73 +victor brown 25.427272727272726 +victor brown 26.218571428571433 +victor davidson 20.55 +victor davidson 22.21666666666667 +victor davidson 29.778 +victor ellison 13.0775 +victor ellison 33.666 +victor hernandez 10.896 +victor hernandez 18.922 +victor hernandez 24.908888888888892 +victor hernandez 27.426666666666666 +victor hernandez 35.6675 +victor johnson 20.02 +victor johnson 27.070000000000004 +victor johnson 29.0775 +victor king 18.066666666666666 +victor king 21.488 +victor laertes 26.77777777777778 +victor laertes 28.095000000000002 +victor miller 5.3100000000000005 +victor nixon 21.395714285714288 +victor nixon 28.33 +victor ovid 35.225 +victor polk 21.990000000000002 +victor quirinius 24.62833333333333 +victor quirinius 29.742500000000003 +victor robinson 14.575 +victor robinson 25.92 +victor steinbeck 26.136666666666667 +victor steinbeck 26.485 +victor steinbeck 34.745999999999995 +victor thompson 18.735 +victor van buren 27.758333333333336 +victor van buren 37.38333333333333 +victor white 24.607999999999997 +victor white 30.66 +victor xylophone 2.775 +victor xylophone 8.356666666666667 +victor xylophone 24.259999999999998 +victor xylophone 25.636666666666667 +victor xylophone 31.610000000000003 +victor young 22.264444444444443 +victor zipper 39.84 +wendy allen 3.4 +wendy allen 24.695000000000004 +wendy allen 29.912 +wendy brown 28.22 +wendy brown 36.74 +wendy ellison 17.549999999999997 +wendy ellison 22.720000000000002 +wendy falkner 13.765 +wendy falkner 24.424444444444443 +wendy falkner 27.86733333333333 +wendy garcia 12.3 +wendy garcia 22.396666666666665 +wendy garcia 26.8325 +wendy garcia 28.596666666666664 +wendy hernandez 21.111428571428572 +wendy ichabod 4.44 +wendy king 23.654285714285713 +wendy king 29.325714285714287 +wendy king 34.21666666666667 +wendy laertes 31.160714285714285 +wendy laertes 31.46666666666667 +wendy laertes 39.22 +wendy miller 12.73 +wendy miller 30.343333333333334 +wendy nixon 19.92714285714286 +wendy nixon 29.675714285714285 +wendy ovid 21.193749999999998 +wendy ovid 28.49846153846154 +wendy polk 20.94 +wendy polk 22.999999999999996 +wendy quirinius 21.05 +wendy quirinius 26.8425 +wendy robinson 8.39 +wendy robinson 24.05 +wendy robinson 26.974285714285713 +wendy steinbeck 26.765 +wendy thompson 24.14 +wendy thompson 28.995384615384616 +wendy underhill 23.118333333333336 +wendy underhill 25.581666666666667 +wendy underhill 32.985 +wendy van buren 25.151666666666667 +wendy van buren 27.077142857142857 +wendy white 24.4025 +wendy xylophone 22.85181818181818 +wendy xylophone 26.96 +wendy young 4.83 +wendy young 21.325 +xavier allen 19.133333333333333 +xavier allen 26.11466666666667 +xavier allen 34.58 +xavier brown 2.63 +xavier brown 24.764285714285712 +xavier brown 30.166666666666668 +xavier carson 29.006666666666664 +xavier carson 32.106 +xavier davidson 14.094999999999999 +xavier davidson 15.906666666666666 +xavier davidson 27.353333333333335 +xavier ellison 22.174166666666668 +xavier ellison 35.01 +xavier garcia 30.357500000000005 +xavier hernandez 19.87 +xavier hernandez 20.805 +xavier hernandez 33.497499999999995 +xavier ichabod 12.34 +xavier ichabod 26.166249999999998 +xavier johnson 20.33222222222222 +xavier johnson 22.503333333333334 +xavier king 1.3 +xavier king 31.348571428571425 +xavier laertes 7.420000000000001 +xavier ovid 25.576 +xavier polk 11.094285714285714 +xavier polk 19.93 +xavier polk 23.63125 +xavier polk 30.194 +xavier quirinius 13.776666666666666 +xavier quirinius 22.27 +xavier quirinius 24.977692307692312 +xavier quirinius 34.95 +xavier thompson 16.47 +xavier underhill 1.31 +xavier white 19.331666666666667 +xavier white 34.68 +xavier xylophone 21.09625 +xavier zipper 14.89 +yuri allen 18.490000000000002 +yuri allen 22.689999999999998 +yuri brown 15.502857142857142 +yuri brown 22.934285714285714 +yuri carson 27.139999999999997 +yuri carson 35.27 +yuri ellison 10.52 +yuri ellison 25.2025 +yuri falkner 24.633076923076924 +yuri falkner 28.52 +yuri garcia 25.545 +yuri hernandez 16.35 +yuri johnson 19.9525 +yuri johnson 27.636000000000003 +yuri johnson 39.92 +yuri king 15.450000000000001 +yuri laertes 0.41000000000000003 +yuri laertes 33.15 +yuri nixon 27.795 +yuri nixon 39.145 +yuri polk 0.8 +yuri polk 9.705 +yuri polk 25.513333333333332 +yuri quirinius 16.29 +yuri quirinius 19.254999999999995 +yuri quirinius 37.878 +yuri steinbeck 27.6275 +yuri steinbeck 48.89 +yuri thompson 23.330000000000002 +yuri underhill 20.504444444444445 +yuri underhill 21.66 +yuri white 31.205 +yuri xylophone 18.790000000000003 +zach allen 13.06 +zach brown 19.985 +zach brown 26.52333333333333 +zach brown 34.66 +zach brown 34.972857142857144 +zach brown 37.45399999999999 +zach carson 26.195999999999998 +zach ellison 17.55 +zach falkner 3.42 +zach falkner 16.18 +zach garcia 20.062 +zach garcia 25.935 +zach garcia 28.974285714285717 +zach garcia 35.449999999999996 +zach ichabod 10.59 +zach ichabod 31.691999999999997 +zach king 6.81 +zach king 20.817 +zach king 32.542500000000004 +zach miller 13.23 +zach miller 26.30666666666667 +zach miller 26.73 +zach ovid 21.122500000000002 +zach ovid 26.983999999999998 +zach ovid 33.15 +zach ovid 40.59 +zach quirinius 13.38 +zach robinson 20.451999999999998 +zach steinbeck 20.358333333333334 +zach steinbeck 29.65 +zach thompson 16.45 +zach thompson 21.430000000000003 +zach underhill 31.438333333333333 +zach white 23.111428571428572 +zach xylophone 21.221428571428568 +zach xylophone 23.156666666666666 +zach young 24.72666666666667 +zach zipper 19.878888888888884 +zach zipper 34.84571428571429 +zach zipper 35.36 diff --git a/sql/hive/src/test/resources/golden/windowing_udaf2-0-96659fde37d7a38ea15b367b47f59ce2 b/sql/hive/src/test/resources/golden/windowing_udaf2-0-96659fde37d7a38ea15b367b47f59ce2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/windowing_udaf2-1-b4bdee4908b1cb8e240c549ae5cfe4c0 b/sql/hive/src/test/resources/golden/windowing_udaf2-1-b4bdee4908b1cb8e240c549ae5cfe4c0 new file mode 100644 index 0000000000000..17c31c0f04592 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_udaf2-1-b4bdee4908b1cb8e240c549ae5cfe4c0 @@ -0,0 +1 @@ +130091 130091 diff --git a/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-0-f498cccf82480be03022d2a36f87651e b/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-0-f498cccf82480be03022d2a36f87651e new file mode 100644 index 0000000000000..31b1f85a5eb5a --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-0-f498cccf82480be03022d2a36f87651e @@ -0,0 +1,1049 @@ + 4294967354 + 4294967416 + 4294967457 +alice allen 4294967487 +alice allen 4294967525 +alice allen 4294967531 +alice brown 4294967355 +alice carson 4294967370 +alice davidson 4294967517 +alice falkner 4294967316 +alice garcia 4294967369 +alice hernandez 4294967299 +alice hernandez 4294967314 +alice johnson 4294967424 +alice king 4294967387 +alice king 4294967516 +alice king 4294967546 +alice laertes 4294967519 +alice laertes 8589934835 +alice miller 4294967324 +alice nixon 4294967410 +alice nixon 4294967413 +alice nixon 4294967484 +alice ovid 8589934726 +alice polk 4294967366 +alice quirinius 4294967505 +alice quirinius 4294967549 +alice robinson 4294967445 +alice robinson 4294967502 +alice steinbeck 4294967364 +alice steinbeck 4294967474 +alice steinbeck 4294967549 +alice underhill 4294967441 +alice van buren 4294967428 +alice xylophone 4294967363 +alice xylophone 4294967519 +alice xylophone 8589934832 +alice zipper 4294967380 +alice zipper 4294967520 +alice zipper 8589935026 +bob brown 4294967422 +bob brown 4294967427 +bob brown 4294967431 +bob carson 4294967408 +bob davidson 4294967354 +bob davidson 4294967435 +bob davidson 4294967504 +bob ellison 4294967344 +bob ellison 4294967362 +bob ellison 4294967436 +bob ellison 4294967530 +bob falkner 8589934966 +bob garcia 4294967369 +bob garcia 4294967435 +bob garcia 4294967439 +bob garcia 8589934707 +bob garcia 8589934867 +bob hernandez 4294967500 +bob ichabod 4294967424 +bob king 4294967297 +bob king 4294967539 +bob king 8589934870 +bob laertes 4294967380 +bob laertes 4294967472 +bob miller 4294967349 +bob ovid 4294967395 +bob ovid 4294967400 +bob ovid 4294967401 +bob ovid 4294967512 +bob polk 4294967337 +bob quirinius 4294967346 +bob steinbeck 4294967342 +bob van buren 4294967422 +bob white 4294967362 +bob white 4294967493 +bob xylophone 4294967407 +bob xylophone 4294967465 +bob young 4294967413 +bob zipper 4294967299 +bob zipper 8589934723 +bob zipper 8589934840 +calvin allen 12884902208 +calvin brown 4294967411 +calvin brown 4294967437 +calvin brown 4294967530 +calvin carson 8589934876 +calvin davidson 4294967468 +calvin davidson 8589934837 +calvin ellison 4294967480 +calvin falkner 4294967300 +calvin falkner 4294967305 +calvin falkner 4294967345 +calvin falkner 8589934749 +calvin falkner 8589934840 +calvin falkner 8589934978 +calvin garcia 8589934927 +calvin hernandez 12884902173 +calvin johnson 4294967546 +calvin laertes 4294967431 +calvin laertes 4294967499 +calvin nixon 4294967300 +calvin nixon 4294967412 +calvin nixon 4294967488 +calvin ovid 4294967329 +calvin ovid 4294967349 +calvin ovid 8589934723 +calvin ovid 8589934835 +calvin polk 8589934962 +calvin quirinius 4294967521 +calvin quirinius 4294967532 +calvin robinson 4294967326 +calvin steinbeck 4294967474 +calvin steinbeck 4294967505 +calvin steinbeck 8589934722 +calvin thompson 4294967513 +calvin thompson 8589934700 +calvin underhill 4294967478 +calvin van buren 4294967300 +calvin van buren 4294967508 +calvin white 4294967304 +calvin white 8589934924 +calvin xylophone 4294967376 +calvin xylophone 8589934599 +calvin xylophone 8589934866 +calvin young 4294967342 +calvin young 8589934817 +calvin zipper 12884902359 +calvin zipper 17179869649 +david allen 4294967371 +david allen 4294967381 +david brown 8589934762 +david brown 12884902420 +david davidson 4294967522 +david davidson 8589934819 +david davidson 12884902188 +david davidson 12884902327 +david ellison 4294967463 +david ellison 8589934777 +david ellison 12884902263 +david hernandez 4294967324 +david ichabod 4294967487 +david ichabod 12884902220 +david laertes 12884902107 +david nixon 4294967381 +david ovid 4294967396 +david ovid 4294967443 +david quirinius 4294967457 +david quirinius 4294967530 +david quirinius 12884902194 +david robinson 4294967465 +david robinson 17179869575 +david thompson 4294967361 +david underhill 4294967384 +david underhill 8589934942 +david underhill 12884902357 +david van buren 4294967309 +david van buren 8589934901 +david white 4294967428 +david xylophone 4294967479 +david xylophone 4294967480 +david xylophone 8589934856 +david young 4294967296 +david young 4294967305 +ethan allen 4294967351 +ethan brown 4294967320 +ethan brown 4294967331 +ethan brown 4294967403 +ethan brown 4294967420 +ethan brown 8589934797 +ethan brown 8589934805 +ethan carson 4294967352 +ethan ellison 4294967514 +ethan ellison 8589934887 +ethan falkner 4294967318 +ethan falkner 4294967461 +ethan garcia 4294967310 +ethan hernandez 4294967349 +ethan johnson 8589934738 +ethan king 8589934731 +ethan laertes 4294967422 +ethan laertes 4294967531 +ethan laertes 8589934767 +ethan laertes 8589934806 +ethan laertes 8589934830 +ethan laertes 8589934995 +ethan laertes 12884902063 +ethan miller 4294967352 +ethan nixon 8589935019 +ethan ovid 8589934909 +ethan polk 4294967329 +ethan polk 4294967382 +ethan polk 4294967479 +ethan polk 8589935021 +ethan quirinius 4294967348 +ethan quirinius 4294967501 +ethan quirinius 8589934695 +ethan robinson 4294967353 +ethan robinson 8589935019 +ethan underhill 8589934897 +ethan van buren 4294967511 +ethan white 4294967427 +ethan white 8589934975 +ethan xylophone 8589934956 +ethan zipper 4294967462 +ethan zipper 12884902348 +fred davidson 8589934724 +fred davidson 8589934850 +fred davidson 12884902468 +fred ellison 4294967393 +fred ellison 8589934797 +fred ellison 8589934978 +fred falkner 4294967547 +fred falkner 12884902170 +fred falkner 17179869760 +fred hernandez 8589934833 +fred ichabod 8589934853 +fred ichabod 12884902455 +fred johnson 8589934904 +fred king 8589934651 +fred king 8589934951 +fred laertes 8589934883 +fred miller 12884902228 +fred nixon 4294967297 +fred nixon 4294967375 +fred nixon 4294967514 +fred nixon 12884902182 +fred polk 4294967332 +fred polk 4294967458 +fred polk 4294967507 +fred polk 8589934944 +fred quirinius 8589934894 +fred quirinius 12884902335 +fred robinson 8589934904 +fred steinbeck 4294967329 +fred steinbeck 4294967411 +fred steinbeck 4294967472 +fred underhill 4294967387 +fred van buren 8589934830 +fred van buren 12884902319 +fred van buren 12884902382 +fred van buren 17179869836 +fred white 8589934763 +fred young 4294967485 +fred young 8589934832 +fred zipper 12884902371 +gabriella allen 4294967405 +gabriella allen 12884902509 +gabriella brown 4294967403 +gabriella brown 4294967543 +gabriella carson 8589934950 +gabriella davidson 4294967507 +gabriella ellison 4294967393 +gabriella ellison 12884902284 +gabriella falkner 4294967378 +gabriella falkner 4294967523 +gabriella falkner 12884902338 +gabriella garcia 4294967419 +gabriella hernandez 4294967462 +gabriella hernandez 4294967481 +gabriella ichabod 4294967337 +gabriella ichabod 8589934740 +gabriella ichabod 8589934797 +gabriella ichabod 8589934818 +gabriella ichabod 17179869508 +gabriella king 4294967393 +gabriella king 8589934906 +gabriella laertes 4294967410 +gabriella miller 8589934768 +gabriella ovid 4294967522 +gabriella ovid 8589934895 +gabriella polk 4294967302 +gabriella polk 8589934868 +gabriella steinbeck 4294967435 +gabriella steinbeck 4294967500 +gabriella thompson 4294967412 +gabriella thompson 8589934814 +gabriella thompson 12884902318 +gabriella van buren 4294967470 +gabriella van buren 8589934783 +gabriella white 4294967335 +gabriella young 4294967431 +gabriella young 8589934980 +gabriella zipper 4294967510 +gabriella zipper 8589934792 +holly allen 12884901926 +holly brown 8589934722 +holly brown 8589934857 +holly falkner 8589934849 +holly hernandez 8589934749 +holly hernandez 8589934805 +holly hernandez 8589935056 +holly hernandez 12884902485 +holly ichabod 4294967329 +holly ichabod 8589934754 +holly ichabod 8589934981 +holly johnson 4294967535 +holly johnson 12884902194 +holly johnson 17179869874 +holly king 8589934785 +holly king 8589934939 +holly laertes 12884902333 +holly miller 8589934823 +holly nixon 4294967383 +holly nixon 8589934744 +holly polk 4294967434 +holly polk 8589934782 +holly robinson 12884902369 +holly thompson 4294967339 +holly thompson 12884902395 +holly thompson 17179869547 +holly underhill 8589934913 +holly underhill 8589934924 +holly underhill 12884902376 +holly underhill 12884902412 +holly van buren 4294967539 +holly white 17179869548 +holly white 17179869900 +holly xylophone 8589934846 +holly young 4294967500 +holly young 8589934932 +holly zipper 4294967509 +holly zipper 17179869531 +irene allen 12884902413 +irene brown 4294967428 +irene brown 8589934934 +irene brown 12884902207 +irene carson 8589934797 +irene ellison 8589934732 +irene ellison 8589934773 +irene falkner 4294967404 +irene falkner 4294967548 +irene garcia 4294967323 +irene garcia 8589934887 +irene garcia 12884902479 +irene ichabod 4294967509 +irene ichabod 8589934860 +irene johnson 8589934990 +irene laertes 4294967481 +irene laertes 12884902196 +irene laertes 17179869632 +irene miller 4294967387 +irene nixon 4294967538 +irene nixon 12884902129 +irene nixon 12884902324 +irene ovid 8589934764 +irene ovid 8589934886 +irene ovid 8589934903 +irene polk 4294967465 +irene polk 4294967521 +irene polk 8589934672 +irene polk 8589934842 +irene polk 17179869877 +irene quirinius 8589934875 +irene quirinius 12884902269 +irene quirinius 17179869628 +irene robinson 8589934676 +irene steinbeck 4294967549 +irene thompson 4294967479 +irene underhill 8589934694 +irene underhill 12884902077 +irene van buren 8589934932 +irene van buren 12884902202 +irene xylophone 8589934901 +jessica brown 8589934867 +jessica carson 4294967508 +jessica carson 8589934740 +jessica carson 17179869819 +jessica davidson 4294967384 +jessica davidson 8589934864 +jessica davidson 12884902256 +jessica davidson 12884902321 +jessica ellison 4294967316 +jessica ellison 12884902128 +jessica falkner 8589934980 +jessica garcia 4294967540 +jessica garcia 21474837337 +jessica ichabod 8589934816 +jessica johnson 8589935006 +jessica johnson 12884902222 +jessica miller 8589934898 +jessica nixon 8589934742 +jessica nixon 12884902240 +jessica ovid 8589934830 +jessica ovid 12884902307 +jessica polk 21474837163 +jessica quirinius 8589934701 +jessica quirinius 8589934872 +jessica quirinius 12884902159 +jessica quirinius 12884902276 +jessica robinson 4294967542 +jessica thompson 8589934698 +jessica thompson 12884902232 +jessica underhill 8589934810 +jessica underhill 8589934878 +jessica underhill 17179869479 +jessica van buren 8589934726 +jessica white 12884902155 +jessica white 12884902281 +jessica white 12884902296 +jessica white 12884902314 +jessica white 17179869676 +jessica xylophone 17179869697 +jessica young 17179869859 +jessica young 17179869861 +jessica zipper 4294967372 +jessica zipper 8589934727 +jessica zipper 17179869778 +katie allen 8589934791 +katie brown 17179869660 +katie davidson 12884902181 +katie ellison 12884902184 +katie ellison 12884902355 +katie falkner 8589934911 +katie garcia 8589934683 +katie garcia 12884902046 +katie hernandez 8589934812 +katie ichabod 8589934795 +katie ichabod 8589934862 +katie ichabod 8589934869 +katie king 4294967339 +katie king 4294967421 +katie king 8589934826 +katie miller 8589934829 +katie miller 12884902267 +katie nixon 21474837149 +katie ovid 4294967519 +katie polk 8589934726 +katie polk 12884902291 +katie robinson 17179869645 +katie van buren 8589934722 +katie van buren 17179869441 +katie white 4294967306 +katie white 8589934885 +katie xylophone 12884902193 +katie young 8589934819 +katie young 8589935024 +katie young 12884902058 +katie zipper 4294967354 +katie zipper 12884902310 +luke allen 8589934864 +luke allen 8589934931 +luke allen 8589935059 +luke allen 12884902257 +luke allen 12884902322 +luke brown 8589934779 +luke davidson 4294967354 +luke davidson 12884902360 +luke ellison 12884902183 +luke ellison 21474836998 +luke ellison 21474837060 +luke falkner 8589934772 +luke falkner 17179869561 +luke garcia 4294967304 +luke garcia 21474837157 +luke ichabod 12884902150 +luke ichabod 12884902366 +luke johnson 4294967527 +luke johnson 8589934812 +luke johnson 12884902161 +luke laertes 8589935027 +luke laertes 12884902031 +luke laertes 12884902184 +luke laertes 12884902213 +luke laertes 12884902378 +luke miller 8589934826 +luke ovid 4294967492 +luke ovid 8589934913 +luke polk 8589934837 +luke polk 12884902340 +luke quirinius 8589934855 +luke robinson 4294967307 +luke robinson 17179869711 +luke thompson 4294967521 +luke underhill 8589934829 +luke underhill 12884902299 +luke underhill 21474837138 +luke van buren 8589934852 +luke white 12884902418 +luke xylophone 8589934804 +luke zipper 4294967353 +mike allen 17179869750 +mike brown 17179869735 +mike carson 4294967477 +mike carson 8589934803 +mike carson 17179869855 +mike davidson 12884902377 +mike davidson 17179869841 +mike ellison 8589934833 +mike ellison 12884902165 +mike ellison 12884902513 +mike ellison 17179869587 +mike ellison 17179869824 +mike falkner 4294967301 +mike garcia 4294967398 +mike garcia 8589934800 +mike garcia 12884902292 +mike hernandez 8589934824 +mike hernandez 12884902281 +mike ichabod 4294967494 +mike king 4294967347 +mike king 4294967400 +mike king 12884902363 +mike king 12884902475 +mike king 17179869528 +mike king 17179869592 +mike miller 17179869705 +mike nixon 12884902293 +mike nixon 17179869708 +mike polk 17179869752 +mike polk 21474837097 +mike polk 21474837344 +mike quirinius 12884902240 +mike steinbeck 8589934653 +mike steinbeck 12884902273 +mike steinbeck 12884902301 +mike steinbeck 17179869903 +mike van buren 8589934942 +mike van buren 12884902402 +mike white 12884902485 +mike white 17179869676 +mike white 21474836928 +mike white 25769804626 +mike young 8589934704 +mike young 8589934878 +mike young 17179869685 +mike zipper 4294967501 +mike zipper 17179869582 +mike zipper 25769804400 +nick allen 8589934664 +nick allen 8589934860 +nick brown 21474836962 +nick davidson 4294967357 +nick ellison 12884902066 +nick ellison 17179869779 +nick falkner 8589935020 +nick falkner 12884902433 +nick garcia 8589934885 +nick garcia 17179869635 +nick garcia 17179869681 +nick ichabod 12884902193 +nick ichabod 12884902223 +nick ichabod 12884902252 +nick johnson 17179869591 +nick johnson 17179869702 +nick laertes 8589934919 +nick miller 12884902419 +nick nixon 8589934910 +nick ovid 12884902267 +nick polk 17179869712 +nick quirinius 4294967296 +nick quirinius 12884902183 +nick robinson 17179869506 +nick robinson 17179869731 +nick steinbeck 4294967355 +nick thompson 8589934922 +nick underhill 25769804624 +nick van buren 8589934635 +nick xylophone 12884902279 +nick young 12884902399 +nick young 21474837140 +nick zipper 12884902300 +nick zipper 17179869849 +oscar allen 17179869779 +oscar brown 12884902062 +oscar carson 12884902232 +oscar carson 17179869663 +oscar carson 17179869779 +oscar carson 21474837066 +oscar carson 21474837089 +oscar davidson 17179869895 +oscar ellison 4294967304 +oscar ellison 8589934740 +oscar falkner 4294967526 +oscar garcia 21474837156 +oscar hernandez 4294967343 +oscar hernandez 8589935049 +oscar ichabod 8589934837 +oscar ichabod 21474836952 +oscar ichabod 21474837021 +oscar ichabod 25769804491 +oscar johnson 12884902182 +oscar johnson 30064772044 +oscar king 12884902159 +oscar king 17179869738 +oscar king 17179869834 +oscar laertes 4294967550 +oscar laertes 8589934727 +oscar laertes 12884902043 +oscar laertes 12884902478 +oscar nixon 17179869458 +oscar ovid 12884902128 +oscar ovid 12884902240 +oscar ovid 25769804460 +oscar polk 21474836829 +oscar polk 21474837063 +oscar quirinius 8589934728 +oscar quirinius 17179869698 +oscar quirinius 21474837051 +oscar quirinius 25769804521 +oscar robinson 8589934656 +oscar robinson 12884902249 +oscar robinson 21474837105 +oscar robinson 25769804694 +oscar steinbeck 4294967548 +oscar thompson 8589934776 +oscar thompson 12884902164 +oscar thompson 12884902317 +oscar thompson 17179869884 +oscar underhill 8589934895 +oscar van buren 4294967500 +oscar van buren 8589934984 +oscar van buren 21474837205 +oscar white 4294967454 +oscar white 8589934826 +oscar white 21474836931 +oscar white 21474837305 +oscar xylophone 12884902193 +oscar xylophone 12884902307 +oscar xylophone 17179869593 +oscar zipper 8589934865 +oscar zipper 8589934874 +oscar zipper 8589934911 +priscilla brown 8589934848 +priscilla brown 8589935013 +priscilla brown 17179869801 +priscilla carson 12884902145 +priscilla carson 21474836880 +priscilla carson 30064772126 +priscilla ichabod 4294967547 +priscilla ichabod 17179869756 +priscilla johnson 4294967468 +priscilla johnson 8589934667 +priscilla johnson 17179869667 +priscilla johnson 17179869787 +priscilla johnson 25769804279 +priscilla king 12884902153 +priscilla nixon 12884902188 +priscilla nixon 25769804766 +priscilla ovid 12884902234 +priscilla ovid 30064772049 +priscilla polk 17179869480 +priscilla quirinius 12884902171 +priscilla thompson 25769804637 +priscilla underhill 4294967333 +priscilla underhill 17179869740 +priscilla van buren 12884902324 +priscilla van buren 21474837167 +priscilla van buren 21474837343 +priscilla white 4294967419 +priscilla xylophone 8589934792 +priscilla xylophone 12884902245 +priscilla xylophone 12884902287 +priscilla young 21474836992 +priscilla young 34359739656 +priscilla zipper 12884902296 +priscilla zipper 12884902537 +quinn allen 4294967542 +quinn allen 17179869552 +quinn brown 12884902251 +quinn brown 17179869401 +quinn brown 17179869626 +quinn davidson 8589934992 +quinn davidson 17179869690 +quinn davidson 25769804455 +quinn davidson 30064771771 +quinn ellison 12884902376 +quinn ellison 34359739559 +quinn garcia 8589934828 +quinn garcia 12884902387 +quinn garcia 12884902460 +quinn garcia 21474837066 +quinn ichabod 30064772171 +quinn king 4294967458 +quinn king 4294967538 +quinn laertes 8589935080 +quinn laertes 17179869711 +quinn laertes 21474837142 +quinn nixon 17179869672 +quinn ovid 17179869695 +quinn quirinius 21474836827 +quinn robinson 12884902445 +quinn steinbeck 17179869739 +quinn steinbeck 21474836905 +quinn thompson 17179869645 +quinn thompson 25769804317 +quinn underhill 8589934815 +quinn underhill 12884902185 +quinn underhill 30064771762 +quinn van buren 4294967362 +quinn young 8589934731 +quinn zipper 12884902453 +quinn zipper 17179869841 +rachel allen 8589934882 +rachel allen 12884902208 +rachel brown 8589934768 +rachel brown 12884902075 +rachel brown 17179869910 +rachel brown 17179869911 +rachel brown 21474837280 +rachel carson 8589934728 +rachel carson 17179869970 +rachel davidson 30064771666 +rachel ellison 4294967423 +rachel falkner 4294967348 +rachel falkner 12884902482 +rachel falkner 21474837331 +rachel falkner 25769804739 +rachel johnson 38654707197 +rachel king 12884902157 +rachel king 30064771759 +rachel laertes 17179869678 +rachel laertes 25769804379 +rachel ovid 12884902055 +rachel ovid 17179869857 +rachel polk 12884902391 +rachel quirinius 17179869456 +rachel robinson 17179869499 +rachel robinson 17179869703 +rachel robinson 25769804290 +rachel thompson 17179869910 +rachel thompson 21474836989 +rachel thompson 21474837392 +rachel underhill 8589934862 +rachel white 17179869585 +rachel white 21474837039 +rachel young 17179869708 +rachel zipper 4294967434 +rachel zipper 21474837228 +sarah carson 4294967319 +sarah carson 17179869688 +sarah carson 30064772084 +sarah ellison 4294967542 +sarah falkner 17179869797 +sarah falkner 21474837349 +sarah garcia 8589934733 +sarah garcia 8589934858 +sarah garcia 17179869599 +sarah ichabod 12884902196 +sarah ichabod 12884902401 +sarah johnson 12884902455 +sarah johnson 21474836981 +sarah johnson 21474837145 +sarah johnson 25769804480 +sarah king 12884902453 +sarah king 21474837191 +sarah miller 8589934958 +sarah ovid 21474837184 +sarah robinson 21474837237 +sarah robinson 21474837389 +sarah steinbeck 21474837313 +sarah white 17179869905 +sarah white 25769804341 +sarah xylophone 12884902207 +sarah young 21474837319 +sarah zipper 25769804616 +tom brown 8589934894 +tom brown 21474837024 +tom carson 4294967388 +tom carson 12884902278 +tom carson 21474836983 +tom davidson 8589934895 +tom ellison 12884902192 +tom ellison 17179869965 +tom ellison 25769804262 +tom falkner 12884902272 +tom falkner 17179869815 +tom hernandez 4294967296 +tom hernandez 12884902109 +tom ichabod 17179869628 +tom johnson 25769804829 +tom johnson 30064771891 +tom king 12884902390 +tom laertes 12884902181 +tom laertes 12884902236 +tom miller 12884901992 +tom miller 17179869647 +tom miller 21474837107 +tom nixon 17179869677 +tom ovid 12884902279 +tom polk 8589934748 +tom polk 8589934892 +tom quirinius 12884902174 +tom quirinius 21474836986 +tom robinson 8589934753 +tom robinson 12884902203 +tom robinson 12884902358 +tom robinson 21474836952 +tom steinbeck 8589934912 +tom van buren 8589934823 +tom van buren 12884902122 +tom van buren 25769804641 +tom white 21474837076 +tom young 4294967535 +tom young 21474837038 +tom zipper 30064772355 +ulysses brown 8589934991 +ulysses carson 8589934789 +ulysses carson 21474837258 +ulysses carson 25769804457 +ulysses carson 34359739082 +ulysses davidson 12884902216 +ulysses ellison 17179869551 +ulysses garcia 12884902382 +ulysses hernandez 12884902210 +ulysses hernandez 12884902276 +ulysses hernandez 17179869748 +ulysses ichabod 4294967353 +ulysses ichabod 12884902217 +ulysses johnson 21474837122 +ulysses king 8589934995 +ulysses laertes 8589934801 +ulysses laertes 21474837354 +ulysses laertes 25769804499 +ulysses miller 21474837284 +ulysses miller 30064771926 +ulysses nixon 17179869288 +ulysses ovid 17179869754 +ulysses polk 8589934855 +ulysses polk 8589934862 +ulysses polk 12884902420 +ulysses polk 17179869479 +ulysses quirinius 17179869659 +ulysses robinson 4294967531 +ulysses steinbeck 8589935027 +ulysses steinbeck 21474837100 +ulysses thompson 12884902194 +ulysses underhill 8589934760 +ulysses underhill 8589934799 +ulysses underhill 12884902240 +ulysses underhill 17179869759 +ulysses underhill 17179869760 +ulysses underhill 17179869939 +ulysses underhill 21474837264 +ulysses van buren 8589934938 +ulysses white 25769804453 +ulysses white 30064772086 +ulysses xylophone 8589935029 +ulysses xylophone 12884902249 +ulysses xylophone 25769804765 +ulysses young 4294967427 +ulysses young 17179869391 +ulysses young 30064771844 +victor allen 8589934793 +victor allen 12884902264 +victor brown 4294967455 +victor brown 17179869657 +victor brown 21474837426 +victor brown 30064771922 +victor davidson 17179869715 +victor davidson 17179869872 +victor davidson 25769804287 +victor ellison 17179869611 +victor ellison 17179869709 +victor hernandez 8589934847 +victor hernandez 12884902463 +victor hernandez 17179869647 +victor hernandez 17179869720 +victor hernandez 25769804310 +victor johnson 17179869652 +victor johnson 21474837148 +victor johnson 25769804771 +victor king 8589934917 +victor king 25769804714 +victor laertes 12884902188 +victor laertes 21474837186 +victor miller 21474837170 +victor nixon 8589934778 +victor nixon 12884902261 +victor ovid 12884902350 +victor polk 17179869376 +victor quirinius 21474837074 +victor quirinius 21474837279 +victor robinson 21474836948 +victor robinson 21474837097 +victor steinbeck 12884902162 +victor steinbeck 17179869721 +victor steinbeck 21474836916 +victor thompson 25769804395 +victor van buren 21474837010 +victor van buren 25769804601 +victor white 8589934816 +victor white 30064771798 +victor xylophone 17179869560 +victor xylophone 25769804719 +victor xylophone 25769804760 +victor xylophone 34359739093 +victor xylophone 34359739095 +victor young 21474837052 +victor zipper 12884902345 +wendy allen 21474837127 +wendy allen 25769804525 +wendy allen 25769804732 +wendy brown 12884902342 +wendy brown 21474836889 +wendy ellison 12884902392 +wendy ellison 21474836763 +wendy falkner 8589934926 +wendy falkner 17179869470 +wendy falkner 25769804816 +wendy garcia 17179869439 +wendy garcia 17179869732 +wendy garcia 30064771654 +wendy garcia 30064771704 +wendy hernandez 17179869752 +wendy ichabod 17179869547 +wendy king 17179869612 +wendy king 21474837301 +wendy king 30064772042 +wendy laertes 8589934872 +wendy laertes 12884902469 +wendy laertes 21474837084 +wendy miller 17179869661 +wendy miller 17179869682 +wendy nixon 12884902521 +wendy nixon 21474836846 +wendy ovid 21474837025 +wendy ovid 38654706512 +wendy polk 8589934960 +wendy polk 21474837144 +wendy quirinius 12884902263 +wendy quirinius 17179869652 +wendy robinson 21474837104 +wendy robinson 25769804321 +wendy robinson 25769804728 +wendy steinbeck 12884902299 +wendy thompson 17179869494 +wendy thompson 21474837072 +wendy underhill 17179869898 +wendy underhill 21474837064 +wendy underhill 25769804845 +wendy van buren 25769804447 +wendy van buren 25769804679 +wendy white 17179869866 +wendy xylophone 17179869596 +wendy xylophone 25769804554 +wendy young 4294967313 +wendy young 25769804562 +xavier allen 12884902364 +xavier allen 17179869960 +xavier allen 21474836864 +xavier brown 8589934824 +xavier brown 17179869646 +xavier brown 25769804653 +xavier carson 17179869770 +xavier carson 21474837445 +xavier davidson 30064772118 +xavier davidson 34359739403 +xavier davidson 38654706539 +xavier ellison 34359739490 +xavier ellison 34359739559 +xavier garcia 21474837142 +xavier hernandez 21474837012 +xavier hernandez 25769804421 +xavier hernandez 38654707021 +xavier ichabod 12884902315 +xavier ichabod 17179869567 +xavier johnson 8589934922 +xavier johnson 38654707066 +xavier king 12884902272 +xavier king 21474836962 +xavier laertes 17179869795 +xavier ovid 17179869597 +xavier polk 12884902254 +xavier polk 17179869581 +xavier polk 17179869743 +xavier polk 34359739344 +xavier quirinius 12884902240 +xavier quirinius 21474836996 +xavier quirinius 25769804437 +xavier quirinius 25769804456 +xavier thompson 17179869822 +xavier underhill 8589934813 +xavier white 12884902262 +xavier white 12884902366 +xavier xylophone 17179869722 +xavier zipper 12884902377 +yuri allen 8589935035 +yuri allen 12884902279 +yuri brown 8589934912 +yuri brown 12884902319 +yuri carson 21474837146 +yuri carson 25769804245 +yuri ellison 25769804504 +yuri ellison 25769804568 +yuri falkner 25769804699 +yuri falkner 42949674720 +yuri garcia 4294967362 +yuri hernandez 21474837117 +yuri johnson 21474837002 +yuri johnson 21474837165 +yuri johnson 25769804545 +yuri king 30064772090 +yuri laertes 30064772076 +yuri laertes 34359739328 +yuri nixon 12884902232 +yuri nixon 12884902265 +yuri polk 12884902362 +yuri polk 21474837245 +yuri polk 25769804539 +yuri quirinius 12884902198 +yuri quirinius 17179869606 +yuri quirinius 30064771819 +yuri steinbeck 4294967535 +yuri steinbeck 8589934657 +yuri thompson 12884902467 +yuri underhill 17179869566 +yuri underhill 17179869715 +yuri white 34359739045 +yuri xylophone 12884902412 +zach allen 17179869908 +zach brown 21474836879 +zach brown 21474836891 +zach brown 21474837040 +zach brown 21474837073 +zach brown 30064771852 +zach carson 21474837185 +zach ellison 8589934898 +zach falkner 17179869807 +zach falkner 25769804634 +zach garcia 17179869536 +zach garcia 21474837142 +zach garcia 30064772246 +zach garcia 34359739192 +zach ichabod 17179869613 +zach ichabod 17179869838 +zach king 17179869700 +zach king 21474837427 +zach king 34359739578 +zach miller 4294967391 +zach miller 12884902310 +zach miller 17179869709 +zach ovid 17179869731 +zach ovid 21474837032 +zach ovid 21474837127 +zach ovid 30064771625 +zach quirinius 34359739151 +zach robinson 21474836938 +zach steinbeck 17179869667 +zach steinbeck 25769804623 +zach thompson 12884902354 +zach thompson 17179869659 +zach underhill 12884902149 +zach white 25769804490 +zach xylophone 12884902198 +zach xylophone 21474837163 +zach young 17179869687 +zach zipper 17179869708 +zach zipper 17179869834 +zach zipper 21474837369 diff --git a/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-1-6378faf36ffd3f61e61cee6c0cb70e6 b/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-1-6378faf36ffd3f61e61cee6c0cb70e6 new file mode 100644 index 0000000000000..1436509e4ec17 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-1-6378faf36ffd3f61e61cee6c0cb70e6 @@ -0,0 +1,1049 @@ + 9.220000267028809 + 43.72999954223633 + 89.52999877929688 +alice allen 2.7899999618530273 +alice allen 21.450000762939453 +alice allen 73.62999725341797 +alice brown 71.30999755859375 +alice carson 39.029998779296875 +alice davidson 70.3499984741211 +alice falkner 90.25 +alice garcia 48.45000076293945 +alice hernandez 88.16999816894531 +alice hernandez 90.55999755859375 +alice johnson 4.46999979019165 +alice king 19.139999389648438 +alice king 23.170000076293945 +alice king 52.22999954223633 +alice laertes 68.94999694824219 +alice laertes 69.52999877929688 +alice miller 68.95999908447266 +alice nixon 40.0 +alice nixon 48.150001525878906 +alice nixon 79.83000183105469 +alice ovid 9.039999961853027 +alice polk 62.900001525878906 +alice quirinius 37.13999938964844 +alice quirinius 62.29999923706055 +alice robinson 5.079999923706055 +alice robinson 56.099998474121094 +alice steinbeck 38.619998931884766 +alice steinbeck 55.5099983215332 +alice steinbeck 92.37000274658203 +alice underhill 98.18000030517578 +alice van buren 38.939998626708984 +alice xylophone 33.58000183105469 +alice xylophone 43.15999984741211 +alice xylophone 78.20999908447266 +alice zipper 26.43000030517578 +alice zipper 42.47999954223633 +alice zipper 89.93000030517578 +bob brown 8.069999694824219 +bob brown 70.93000030517578 +bob brown 93.08999633789062 +bob carson 50.09000015258789 +bob davidson 1.2899999618530273 +bob davidson 53.93000030517578 +bob davidson 74.72000122070312 +bob ellison 41.34000015258789 +bob ellison 65.0199966430664 +bob ellison 75.02999877929688 +bob ellison 80.30000305175781 +bob falkner 16.989999771118164 +bob garcia 4.460000038146973 +bob garcia 5.400000095367432 +bob garcia 45.59000015258789 +bob garcia 80.30000305175781 +bob garcia 87.56999969482422 +bob hernandez 22.68000030517578 +bob ichabod 82.55999755859375 +bob king 8.789999961853027 +bob king 12.539999961853027 +bob king 39.0099983215332 +bob laertes 0.7900000214576721 +bob laertes 10.670000076293945 +bob miller 61.91999816894531 +bob ovid 46.86000061035156 +bob ovid 62.849998474121094 +bob ovid 88.77999877929688 +bob ovid 97.08999633789062 +bob polk 7.980000019073486 +bob quirinius 46.099998474121094 +bob steinbeck 9.699999809265137 +bob van buren 33.66999816894531 +bob white 45.34000015258789 +bob white 45.349998474121094 +bob xylophone 19.690000534057617 +bob xylophone 107.93000221252441 +bob young 35.16999816894531 +bob zipper 1.25 +bob zipper 3.819999933242798 +bob zipper 34.349998474121094 +calvin allen 63.119998931884766 +calvin brown 28.110000610351562 +calvin brown 85.9000015258789 +calvin brown 90.19999694824219 +calvin carson 80.2300033569336 +calvin davidson 31.770000457763672 +calvin davidson 85.51000213623047 +calvin ellison 26.489999771118164 +calvin falkner 2.9700000286102295 +calvin falkner 56.040000915527344 +calvin falkner 56.33000183105469 +calvin falkner 80.5999984741211 +calvin falkner 93.61000061035156 +calvin falkner 94.30999755859375 +calvin garcia 41.849998474121094 +calvin hernandez 33.869998931884766 +calvin johnson 66.61000061035156 +calvin laertes 23.1299991607666 +calvin laertes 50.310001373291016 +calvin nixon 9.8100004196167 +calvin nixon 41.20000076293945 +calvin nixon 69.73999786376953 +calvin ovid 69.95999908447266 +calvin ovid 71.26000213623047 +calvin ovid 79.12000274658203 +calvin ovid 84.72000122070312 +calvin polk 65.72000122070312 +calvin quirinius 29.540000915527344 +calvin quirinius 53.02000045776367 +calvin robinson 40.439998626708984 +calvin steinbeck 15.220000267028809 +calvin steinbeck 22.850000381469727 +calvin steinbeck 93.30000305175781 +calvin thompson 8.90999984741211 +calvin thompson 93.7300033569336 +calvin underhill 59.70000076293945 +calvin van buren 34.209999084472656 +calvin van buren 64.0 +calvin white 50.279998779296875 +calvin white 90.69000244140625 +calvin xylophone 21.700000762939453 +calvin xylophone 25.420000076293945 +calvin xylophone 56.810001373291016 +calvin young 24.489999771118164 +calvin young 39.810001373291016 +calvin zipper 9.1899995803833 +calvin zipper 95.37999725341797 +david allen 51.25 +david allen 64.87000274658203 +david brown 3.2100000381469727 +david brown 93.63999938964844 +david davidson 1.0800000429153442 +david davidson 62.720001220703125 +david davidson 74.1500015258789 +david davidson 95.80999755859375 +david ellison 47.689998626708984 +david ellison 85.2300033569336 +david ellison 94.1500015258789 +david hernandez 99.91000366210938 +david ichabod 5.28000020980835 +david ichabod 82.55000305175781 +david laertes 76.70999908447266 +david nixon 50.31999969482422 +david ovid 25.110000610351562 +david ovid 61.70000076293945 +david quirinius 20.639999389648438 +david quirinius 29.239999771118164 +david quirinius 79.97000122070312 +david robinson 25.280000686645508 +david robinson 161.1199951171875 +david thompson 80.89999771118164 +david underhill 8.319999694824219 +david underhill 88.7699966430664 +david underhill 97.55999755859375 +david van buren 83.56999969482422 +david van buren 115.43999862670898 +david white 45.189998626708984 +david xylophone 8.069999694824219 +david xylophone 54.34000015258789 +david xylophone 72.9800033569336 +david young 10.25 +david young 35.650001525878906 +ethan allen 32.75 +ethan brown 7.110000133514404 +ethan brown 10.09000015258789 +ethan brown 15.630000114440918 +ethan brown 61.86000061035156 +ethan brown 73.18000030517578 +ethan brown 82.30000305175781 +ethan carson 76.33000183105469 +ethan ellison 0.2800000011920929 +ethan ellison 81.47000122070312 +ethan falkner 50.02000045776367 +ethan falkner 59.43000030517578 +ethan garcia 43.189998626708984 +ethan hernandez 49.779998779296875 +ethan johnson 90.05000305175781 +ethan king 4.349999904632568 +ethan laertes 15.449999809265137 +ethan laertes 54.75 +ethan laertes 59.209999084472656 +ethan laertes 70.38999938964844 +ethan laertes 80.70999908447266 +ethan laertes 95.06999969482422 +ethan laertes 96.29000091552734 +ethan miller 25.3700008392334 +ethan nixon 37.779998779296875 +ethan ovid 57.290000915527344 +ethan polk 2.3499999046325684 +ethan polk 21.31999969482422 +ethan polk 23.440000534057617 +ethan polk 122.71999740600586 +ethan quirinius 3.859999895095825 +ethan quirinius 51.84000015258789 +ethan quirinius 97.23999786376953 +ethan robinson 67.94000244140625 +ethan robinson 78.62000274658203 +ethan underhill 55.630001068115234 +ethan van buren 36.70000076293945 +ethan white 60.849998474121094 +ethan white 63.41999816894531 +ethan xylophone 57.11000061035156 +ethan zipper 2.9200000762939453 +ethan zipper 97.51000213623047 +fred davidson 18.860000610351562 +fred davidson 37.2400016784668 +fred davidson 78.30999755859375 +fred ellison 31.179998874664307 +fred ellison 48.59000015258789 +fred ellison 96.77999877929688 +fred falkner 10.289999961853027 +fred falkner 72.04000091552734 +fred falkner 85.0 +fred hernandez 55.9900016784668 +fred ichabod 47.359999656677246 +fred ichabod 81.31999969482422 +fred johnson 96.08999633789062 +fred king 48.369998931884766 +fred king 72.13999843597412 +fred laertes 57.63999938964844 +fred miller 46.970001220703125 +fred nixon 28.690000534057617 +fred nixon 38.04999923706055 +fred nixon 70.5199966430664 +fred nixon 93.02999877929688 +fred polk 23.959999084472656 +fred polk 39.18000030517578 +fred polk 47.31999969482422 +fred polk 90.12000274658203 +fred quirinius 15.300000190734863 +fred quirinius 29.399999618530273 +fred robinson 89.02999877929688 +fred steinbeck 32.22999954223633 +fred steinbeck 41.310001373291016 +fred steinbeck 91.05000305175781 +fred underhill 90.7699966430664 +fred van buren 1.0199999809265137 +fred van buren 21.940000534057617 +fred van buren 52.869998931884766 +fred van buren 83.58000183105469 +fred white 37.79999923706055 +fred young 46.79999923706055 +fred young 97.70999908447266 +fred zipper 29.020000457763672 +gabriella allen 46.27000045776367 +gabriella allen 64.22000122070312 +gabriella brown 15.260000228881836 +gabriella brown 84.83000183105469 +gabriella carson 42.7599983215332 +gabriella davidson 6.550000190734863 +gabriella ellison 48.08000183105469 +gabriella ellison 71.54000091552734 +gabriella falkner 10.170000076293945 +gabriella falkner 51.720001220703125 +gabriella falkner 87.61000061035156 +gabriella garcia 43.0099983215332 +gabriella hernandez 76.91999816894531 +gabriella hernandez 92.9800033569336 +gabriella ichabod 10.729999542236328 +gabriella ichabod 26.639999389648438 +gabriella ichabod 66.36000061035156 +gabriella ichabod 71.12999725341797 +gabriella ichabod 90.3499984741211 +gabriella king 20.670000076293945 +gabriella king 80.45999908447266 +gabriella laertes 65.37999725341797 +gabriella miller 50.83000183105469 +gabriella ovid 77.7400016784668 +gabriella ovid 92.4000015258789 +gabriella polk 35.68000030517578 +gabriella polk 88.05000305175781 +gabriella steinbeck 46.45000076293945 +gabriella steinbeck 78.63999938964844 +gabriella thompson 73.31999969482422 +gabriella thompson 88.36000061035156 +gabriella thompson 94.25 +gabriella van buren 69.80000305175781 +gabriella van buren 70.05999755859375 +gabriella white 55.18000030517578 +gabriella young 9.25 +gabriella young 59.709999084472656 +gabriella zipper 36.2599983215332 +gabriella zipper 91.62999725341797 +holly allen 44.56999969482422 +holly brown 77.80999755859375 +holly brown 78.7300033569336 +holly falkner 80.73999786376953 +holly hernandez 20.81999969482422 +holly hernandez 21.190000534057617 +holly hernandez 24.790000915527344 +holly hernandez 30.25 +holly ichabod 83.2699966430664 +holly ichabod 84.69000244140625 +holly ichabod 90.51000213623047 +holly johnson 36.95000076293945 +holly johnson 64.36000061035156 +holly johnson 65.62000274658203 +holly king 42.310001373291016 +holly king 55.38999938964844 +holly laertes 52.5 +holly miller 50.400001525878906 +holly nixon 53.779998779296875 +holly nixon 88.0199966430664 +holly polk 3.619999885559082 +holly polk 98.30999755859375 +holly robinson 69.31999969482422 +holly thompson 0.07999999821186066 +holly thompson 86.69000244140625 +holly thompson 145.93999481201172 +holly underhill 42.54999923706055 +holly underhill 50.40999984741211 +holly underhill 79.95999908447266 +holly underhill 96.68000030517578 +holly van buren 68.80999755859375 +holly white 7.960000038146973 +holly white 32.91999816894531 +holly xylophone 93.11000061035156 +holly young 60.220001220703125 +holly young 66.16999816894531 +holly zipper 99.12999725341797 +holly zipper 99.29000091552734 +irene allen 38.849998474121094 +irene brown 4.789999961853027 +irene brown 53.939998626708984 +irene brown 87.66999816894531 +irene carson 94.54000091552734 +irene ellison 45.2400016784668 +irene ellison 50.08000183105469 +irene falkner 22.079999923706055 +irene falkner 99.91999816894531 +irene garcia 15.369999885559082 +irene garcia 58.43000030517578 +irene garcia 86.93000030517578 +irene ichabod 41.439998626708984 +irene ichabod 99.62000274658203 +irene johnson 5.880000114440918 +irene laertes 9.569999694824219 +irene laertes 42.66999816894531 +irene laertes 44.43000030517578 +irene miller 65.44000244140625 +irene nixon 15.100000381469727 +irene nixon 29.780000686645508 +irene nixon 42.560001373291016 +irene ovid 5.239999771118164 +irene ovid 35.130001068115234 +irene ovid 79.75 +irene polk 0.9800000190734863 +irene polk 24.020000457763672 +irene polk 42.2400016784668 +irene polk 47.08000183105469 +irene polk 95.83999633789062 +irene quirinius 12.899999618530273 +irene quirinius 58.86000061035156 +irene quirinius 70.0 +irene robinson 94.2699966430664 +irene steinbeck 94.33000183105469 +irene thompson 78.30000305175781 +irene underhill 28.309999465942383 +irene underhill 57.349998474121094 +irene van buren 54.439998626708984 +irene van buren 54.9900016784668 +irene xylophone 74.19000244140625 +jessica brown 51.290000915527344 +jessica carson 25.549999237060547 +jessica carson 31.860000610351562 +jessica carson 62.20000076293945 +jessica davidson 33.54999923706055 +jessica davidson 49.77000045776367 +jessica davidson 95.33999633789062 +jessica davidson 99.20999908447266 +jessica ellison 11.180000305175781 +jessica ellison 22.780000686645508 +jessica falkner 99.6500015258789 +jessica garcia 5.539999961853027 +jessica garcia 87.92999941110611 +jessica ichabod 59.15999984741211 +jessica johnson 9.5600004196167 +jessica johnson 40.79999923706055 +jessica miller 151.0199966430664 +jessica nixon 77.0999984741211 +jessica nixon 90.06999969482422 +jessica ovid 71.68000030517578 +jessica ovid 119.9000015258789 +jessica polk 49.68000030517578 +jessica quirinius 22.940000534057617 +jessica quirinius 32.470001220703125 +jessica quirinius 35.619998931884766 +jessica quirinius 46.869998931884766 +jessica robinson 112.36000442504883 +jessica thompson 38.33000183105469 +jessica thompson 89.55000305175781 +jessica underhill 26.079999923706055 +jessica underhill 45.41999816894531 +jessica underhill 46.209999084472656 +jessica van buren 9.739999771118164 +jessica white 11.550000190734863 +jessica white 36.58000183105469 +jessica white 73.93000030517578 +jessica white 74.30000305175781 +jessica white 96.62000274658203 +jessica xylophone 53.060001373291016 +jessica young 11.1899995803833 +jessica young 43.369998931884766 +jessica zipper 6.630000114440918 +jessica zipper 12.020000457763672 +jessica zipper 92.43999862670898 +katie allen 64.66999816894531 +katie brown 27.719999313354492 +katie davidson 170.84000396728516 +katie ellison 3.609999895095825 +katie ellison 80.97000122070312 +katie falkner 18.5 +katie garcia 24.729999542236328 +katie garcia 84.4000015258789 +katie hernandez 38.61999988555908 +katie ichabod 30.709999084472656 +katie ichabod 39.97999954223633 +katie ichabod 43.16999816894531 +katie king 39.34000015258789 +katie king 39.83000183105469 +katie king 97.80999755859375 +katie miller 31.399999618530273 +katie miller 74.77999877929688 +katie nixon 121.3700008392334 +katie ovid 50.65999984741211 +katie polk 11.680000305175781 +katie polk 40.2400016784668 +katie robinson 13.890000343322754 +katie van buren 17.739999771118164 +katie van buren 52.529998779296875 +katie white 1.309999942779541 +katie white 34.72999954223633 +katie xylophone 14.130000114440918 +katie young 31.010000228881836 +katie young 72.51000213623047 +katie young 97.56999969482422 +katie zipper 18.93000030517578 +katie zipper 58.75 +luke allen 15.180000305175781 +luke allen 50.959999084472656 +luke allen 66.61000061035156 +luke allen 89.55000305175781 +luke allen 99.38999938964844 +luke brown 51.790000915527344 +luke davidson 7.050000190734863 +luke davidson 28.950000762939453 +luke ellison 1.8700000047683716 +luke ellison 16.25 +luke ellison 87.83000183105469 +luke falkner 32.25 +luke falkner 39.60000038146973 +luke garcia 13.350000381469727 +luke garcia 30.3700008392334 +luke ichabod 8.449999809265137 +luke ichabod 97.87000274658203 +luke johnson 11.149999618530273 +luke johnson 14.4399995803833 +luke johnson 31.670000076293945 +luke laertes 0.5199999809265137 +luke laertes 4.800000190734863 +luke laertes 11.819999694824219 +luke laertes 16.690000534057617 +luke laertes 45.9900016784668 +luke miller 97.6500015258789 +luke ovid 38.04999923706055 +luke ovid 159.68000030517578 +luke polk 46.880001068115234 +luke polk 95.27999877929688 +luke quirinius 40.41999816894531 +luke robinson 55.099998474121094 +luke robinson 65.69999694824219 +luke thompson 94.37999725341797 +luke underhill 59.68000030517578 +luke underhill 95.52999877929688 +luke underhill 96.94000244140625 +luke van buren 148.62999725341797 +luke white 67.12000274658203 +luke xylophone 48.279998779296875 +luke zipper 24.829999923706055 +mike allen 48.53999900817871 +mike brown 48.22999954223633 +mike carson 20.06999969482422 +mike carson 47.56999969482422 +mike carson 81.66000366210938 +mike davidson 27.309999465942383 +mike davidson 54.83000183105469 +mike ellison 28.559999465942383 +mike ellison 37.099998474121094 +mike ellison 62.13999938964844 +mike ellison 79.37999725341797 +mike ellison 85.73999786376953 +mike falkner 16.479999542236328 +mike garcia 70.8499984741211 +mike garcia 75.83000183105469 +mike garcia 79.20999908447266 +mike hernandez 37.900001525878906 +mike hernandez 59.45000076293945 +mike ichabod 64.7699966430664 +mike king 38.790000915527344 +mike king 62.7400016784668 +mike king 78.26000213623047 +mike king 84.2300033569336 +mike king 85.0999984741211 +mike king 94.68000030517578 +mike miller 3.9600000381469727 +mike nixon 60.119998931884766 +mike nixon 92.95999908447266 +mike polk 12.449999809265137 +mike polk 27.06999969482422 +mike polk 99.68000030517578 +mike quirinius 89.37999725341797 +mike steinbeck 5.849999904632568 +mike steinbeck 85.13999938964844 +mike steinbeck 93.07000207901001 +mike steinbeck 97.45999908447266 +mike van buren 80.83999633789062 +mike van buren 114.56999969482422 +mike white 9.569999694824219 +mike white 28.889999389648438 +mike white 32.0099983215332 +mike white 91.87999725341797 +mike young 7.820000171661377 +mike young 74.58999633789062 +mike young 83.54000091552734 +mike zipper 26.729999542236328 +mike zipper 83.91999816894531 +mike zipper 97.38999938964844 +nick allen 21.830000400543213 +nick allen 35.08000183105469 +nick brown 42.5099983215332 +nick davidson 49.439998626708984 +nick ellison 9.680000305175781 +nick ellison 89.01000213623047 +nick falkner 10.130000114440918 +nick falkner 88.47000122070312 +nick garcia 13.9399995803833 +nick garcia 26.389999389648438 +nick garcia 46.43000030517578 +nick ichabod 23.450000762939453 +nick ichabod 47.59000015258789 +nick ichabod 74.41999816894531 +nick johnson 3.9700000286102295 +nick johnson 94.08000183105469 +nick laertes 96.25 +nick miller 82.97000122070312 +nick nixon 96.37999725341797 +nick ovid 87.98999786376953 +nick polk 59.27000141143799 +nick quirinius 67.44999694824219 +nick quirinius 81.16999816894531 +nick robinson 57.66999816894531 +nick robinson 60.709999084472656 +nick steinbeck 97.83000183105469 +nick thompson 11.90999984741211 +nick underhill 20.809999465942383 +nick van buren 51.290000915527344 +nick xylophone 103.45999908447266 +nick young 0.27000001072883606 +nick young 24.799999237060547 +nick zipper 56.619998931884766 +nick zipper 119.0199966430664 +oscar allen 18.6299991607666 +oscar brown 13.100000381469727 +oscar carson 6.869999885559082 +oscar carson 55.20000076293945 +oscar carson 78.9800033569336 +oscar carson 87.4800033569336 +oscar carson 98.51000213623047 +oscar davidson 64.45999908447266 +oscar ellison 57.88999938964844 +oscar ellison 107.7100019454956 +oscar falkner 98.4800033569336 +oscar garcia 67.4800033569336 +oscar hernandez 95.4800033569336 +oscar hernandez 125.92999649047852 +oscar ichabod 3.3299999237060547 +oscar ichabod 33.52000045776367 +oscar ichabod 71.80000305175781 +oscar ichabod 76.69000244140625 +oscar johnson 16.09000015258789 +oscar johnson 139.69000244140625 +oscar king 19.059999465942383 +oscar king 25.8799991607666 +oscar king 59.5 +oscar laertes 5.510000228881836 +oscar laertes 8.420000076293945 +oscar laertes 9.260000228881836 +oscar laertes 27.1200008392334 +oscar nixon 41.619998931884766 +oscar ovid 37.13999938964844 +oscar ovid 82.23999786376953 +oscar ovid 91.52999877929688 +oscar polk 30.610000610351562 +oscar polk 63.900001525878906 +oscar quirinius 41.45000076293945 +oscar quirinius 65.43000030517578 +oscar quirinius 113.35000228881836 +oscar quirinius 139.10000610351562 +oscar robinson 11.34000015258789 +oscar robinson 42.849998474121094 +oscar robinson 74.52999877929688 +oscar robinson 131.31999969482422 +oscar steinbeck 29.59000015258789 +oscar thompson 31.90999984741211 +oscar thompson 41.34000015258789 +oscar thompson 60.529998779296875 +oscar thompson 70.88999938964844 +oscar underhill 87.4000015258789 +oscar van buren 2.180000066757202 +oscar van buren 61.880001068115234 +oscar van buren 91.77999877929688 +oscar white 19.0 +oscar white 28.450000762939453 +oscar white 51.849998474121094 +oscar white 59.83000183105469 +oscar xylophone 21.799999237060547 +oscar xylophone 57.119998931884766 +oscar xylophone 57.22999954223633 +oscar zipper 13.989999771118164 +oscar zipper 32.88999938964844 +oscar zipper 39.81999969482422 +priscilla brown 70.23999786376953 +priscilla brown 80.5199966430664 +priscilla brown 104.63999938964844 +priscilla carson 7.960000038146973 +priscilla carson 79.80999946594238 +priscilla carson 85.43000316619873 +priscilla ichabod 80.04000091552734 +priscilla ichabod 92.61000061035156 +priscilla johnson 61.939998626708984 +priscilla johnson 67.9800033569336 +priscilla johnson 68.32999992370605 +priscilla johnson 91.4800033569336 +priscilla johnson 92.48000144958496 +priscilla king 43.91999816894531 +priscilla nixon 95.80999755859375 +priscilla nixon 107.69000244140625 +priscilla ovid 52.72999954223633 +priscilla ovid 125.73999643325806 +priscilla polk 15.149999618530273 +priscilla quirinius 9.710000038146973 +priscilla thompson 9.800000190734863 +priscilla underhill 35.720001220703125 +priscilla underhill 68.22000122070312 +priscilla van buren 68.88999938964844 +priscilla van buren 91.61000061035156 +priscilla van buren 170.5500030517578 +priscilla white 78.27999877929688 +priscilla xylophone 0.15000000596046448 +priscilla xylophone 21.489999771118164 +priscilla xylophone 59.61000061035156 +priscilla young 0.4300000071525574 +priscilla young 4.320000171661377 +priscilla zipper 18.6299991607666 +priscilla zipper 25.670000076293945 +quinn allen 54.72999954223633 +quinn allen 83.33000183105469 +quinn brown 24.280000686645508 +quinn brown 52.439998626708984 +quinn brown 80.58000183105469 +quinn davidson 61.57999849319458 +quinn davidson 67.18000030517578 +quinn davidson 83.4000015258789 +quinn davidson 95.11000061035156 +quinn ellison 19.280000686645508 +quinn ellison 30.649999618530273 +quinn garcia 40.97999954223633 +quinn garcia 59.9900016784668 +quinn garcia 74.0199966430664 +quinn garcia 172.8499984741211 +quinn ichabod 36.790000915527344 +quinn king 74.62000274658203 +quinn king 86.2300033569336 +quinn laertes 4.710000038146973 +quinn laertes 41.290000915527344 +quinn laertes 76.5199966430664 +quinn nixon 86.64000129699707 +quinn ovid 52.500000953674316 +quinn quirinius 32.18000030517578 +quinn robinson 38.64999866485596 +quinn steinbeck 8.449999809265137 +quinn steinbeck 66.51000213623047 +quinn thompson 74.9399995803833 +quinn thompson 76.27999877929688 +quinn underhill 17.15999984741211 +quinn underhill 79.4800033569336 +quinn underhill 140.92000198364258 +quinn van buren 82.5199966430664 +quinn young 45.060001373291016 +quinn zipper 22.25 +quinn zipper 58.0 +rachel allen 15.8100004196167 +rachel allen 74.44999694824219 +rachel brown 2.9600000381469727 +rachel brown 30.809999465942383 +rachel brown 33.36000061035156 +rachel brown 34.40999984741211 +rachel brown 52.16999816894531 +rachel carson 37.599998474121094 +rachel carson 98.95999908447266 +rachel davidson 4.920000076293945 +rachel ellison 10.600000381469727 +rachel falkner 46.150001525878906 +rachel falkner 80.91999816894531 +rachel falkner 88.80000305175781 +rachel falkner 99.23999786376953 +rachel johnson 62.22999954223633 +rachel king 36.220001220703125 +rachel king 59.45000076293945 +rachel laertes 44.220001220703125 +rachel laertes 45.45000076293945 +rachel ovid 0.6000000238418579 +rachel ovid 1.0800000429153442 +rachel polk 89.27999877929688 +rachel quirinius 12.4399995803833 +rachel robinson 4.570000171661377 +rachel robinson 30.360000610351562 +rachel robinson 64.94999694824219 +rachel thompson 0.5600000023841858 +rachel thompson 4.170000076293945 +rachel thompson 58.52000045776367 +rachel underhill 48.45000076293945 +rachel white 43.709999084472656 +rachel white 94.72000122070312 +rachel young 43.130001068115234 +rachel zipper 7.059999942779541 +rachel zipper 72.18000030517578 +sarah carson 1.909999966621399 +sarah carson 14.210000038146973 +sarah carson 78.88999938964844 +sarah ellison 16.989999771118164 +sarah falkner 90.27999877929688 +sarah falkner 99.36000061035156 +sarah garcia 41.290000915527344 +sarah garcia 58.010000228881836 +sarah garcia 153.8800048828125 +sarah ichabod 81.31999969482422 +sarah ichabod 97.26000213623047 +sarah johnson 16.239999771118164 +sarah johnson 45.099998474121094 +sarah johnson 73.87999725341797 +sarah johnson 77.66000366210938 +sarah king 41.869998931884766 +sarah king 48.25 +sarah miller 41.709999084472656 +sarah ovid 60.02000045776367 +sarah robinson 33.83000183105469 +sarah robinson 66.88999938964844 +sarah steinbeck 40.16999816894531 +sarah white 37.849998474121094 +sarah white 89.80999755859375 +sarah xylophone 68.31999969482422 +sarah young 45.560001373291016 +sarah zipper 83.08000183105469 +tom brown 8.609999656677246 +tom brown 12.319999694824219 +tom carson 5.440000057220459 +tom carson 16.079999923706055 +tom carson 18.889999389648438 +tom davidson 170.0 +tom ellison 76.73999786376953 +tom ellison 98.2300033569336 +tom ellison 155.99999618530273 +tom falkner 60.130001068115234 +tom falkner 88.22000122070312 +tom hernandez 41.36000061035156 +tom hernandez 81.63999938964844 +tom ichabod 103.29000282287598 +tom johnson 14.920000076293945 +tom johnson 43.56999969482422 +tom king 15.75 +tom laertes 43.310001373291016 +tom laertes 64.6500015258789 +tom miller 21.229999542236328 +tom miller 68.25 +tom miller 139.04000091552734 +tom nixon 153.83999633789062 +tom ovid 8.670000076293945 +tom polk 38.29999923706055 +tom polk 54.43000030517578 +tom quirinius 10.1899995803833 +tom quirinius 75.31999969482422 +tom robinson 90.69000244140625 +tom robinson 98.72000122070312 +tom robinson 99.1500015258789 +tom robinson 123.5199966430664 +tom steinbeck 26.489999771118164 +tom van buren 3.2799999713897705 +tom van buren 40.779998779296875 +tom van buren 63.5099983215332 +tom white 40.040000915527344 +tom young 22.850000381469727 +tom young 84.30999755859375 +tom zipper 122.78000259399414 +ulysses brown 72.79000091552734 +ulysses carson 77.41999816894531 +ulysses carson 79.54000091552734 +ulysses carson 146.7100067138672 +ulysses carson 220.18000030517578 +ulysses davidson 55.16999816894531 +ulysses ellison 96.7300033569336 +ulysses garcia 89.80000305175781 +ulysses hernandez 35.16999816894531 +ulysses hernandez 54.470001220703125 +ulysses hernandez 68.25 +ulysses ichabod 19.1299991607666 +ulysses ichabod 98.56999969482422 +ulysses johnson 102.5999984741211 +ulysses king 74.19000244140625 +ulysses laertes 1.9199999570846558 +ulysses laertes 24.860000610351562 +ulysses laertes 50.1899995803833 +ulysses miller 2.9600000381469727 +ulysses miller 76.27999877929688 +ulysses nixon 80.95999908447266 +ulysses ovid 29.360000610351562 +ulysses polk 8.710000038146973 +ulysses polk 60.060001373291016 +ulysses polk 65.0199966430664 +ulysses polk 97.10000038146973 +ulysses quirinius 112.56999969482422 +ulysses robinson 104.85999870300293 +ulysses steinbeck 32.40999984741211 +ulysses steinbeck 74.0 +ulysses thompson 198.83000564575195 +ulysses underhill 14.119999885559082 +ulysses underhill 22.360000610351562 +ulysses underhill 35.88999938964844 +ulysses underhill 57.369998931884766 +ulysses underhill 81.58000183105469 +ulysses underhill 88.4800033569336 +ulysses underhill 99.66999816894531 +ulysses van buren 95.52999877929688 +ulysses white 59.54999923706055 +ulysses white 170.0800018310547 +ulysses xylophone 39.689998626708984 +ulysses xylophone 54.099998474121094 +ulysses xylophone 57.3100004196167 +ulysses young 14.930000305175781 +ulysses young 32.52000045776367 +ulysses young 114.55999946594238 +victor allen 44.27000045776367 +victor allen 89.5 +victor brown 59.34000015258789 +victor brown 77.88999938964844 +victor brown 90.37999725341797 +victor brown 91.97000122070312 +victor davidson 60.2599983215332 +victor davidson 66.5999984741211 +victor davidson 98.54999923706055 +victor ellison 17.8700008392334 +victor ellison 68.8499984741211 +victor hernandez 19.030000686645508 +victor hernandez 59.619998931884766 +victor hernandez 69.87999725341797 +victor hernandez 71.3499984741211 +victor hernandez 74.5199966430664 +victor johnson 18.200000762939453 +victor johnson 42.89000141620636 +victor johnson 72.55999755859375 +victor king 47.880001068115234 +victor king 66.66999816894531 +victor laertes 62.91999816894531 +victor laertes 67.58999633789062 +victor miller 22.1200008392334 +victor nixon 34.029998779296875 +victor nixon 68.5 +victor ovid 125.84000015258789 +victor polk 17.210000038146973 +victor quirinius 50.70000076293945 +victor quirinius 134.4000015258789 +victor robinson 51.560001373291016 +victor robinson 58.66999816894531 +victor steinbeck 12.460000038146973 +victor steinbeck 46.09000015258789 +victor steinbeck 52.720001220703125 +victor thompson 58.65999984741211 +victor van buren 34.970001220703125 +victor van buren 41.68000030517578 +victor white 5.670000076293945 +victor white 135.02999687194824 +victor xylophone 10.09000015258789 +victor xylophone 11.220000267028809 +victor xylophone 28.5 +victor xylophone 62.38999938964844 +victor xylophone 76.0999984741211 +victor young 88.55000305175781 +victor zipper 26.289999961853027 +wendy allen 56.06999969482422 +wendy allen 93.96999740600586 +wendy allen 220.7900003194809 +wendy brown 27.8700008392334 +wendy brown 50.2599983215332 +wendy ellison 94.66000366210938 +wendy ellison 124.93999481201172 +wendy falkner 22.010000228881836 +wendy falkner 97.68000030517578 +wendy falkner 141.36000061035156 +wendy garcia 30.6200008392334 +wendy garcia 57.25 +wendy garcia 82.1500015258789 +wendy garcia 133.3400001525879 +wendy hernandez 48.11000061035156 +wendy ichabod 13.149999618530273 +wendy king 45.189998626708984 +wendy king 63.33000183105469 +wendy king 183.75 +wendy laertes 46.619998931884766 +wendy laertes 70.37999725341797 +wendy laertes 79.98999786376953 +wendy miller 1.2699999809265137 +wendy miller 12.420000076293945 +wendy nixon 45.91999816894531 +wendy nixon 60.2599983215332 +wendy ovid 86.62999725341797 +wendy ovid 95.33000183105469 +wendy polk 32.369998931884766 +wendy polk 42.04000073671341 +wendy quirinius 12.15999984741211 +wendy quirinius 14.300000190734863 +wendy robinson 26.469999313354492 +wendy robinson 71.06999969482422 +wendy robinson 117.02000045776367 +wendy steinbeck 120.67000007629395 +wendy thompson 67.34000015258789 +wendy thompson 85.76000213623047 +wendy underhill 68.04000091552734 +wendy underhill 79.19000244140625 +wendy underhill 89.77999877929688 +wendy van buren 57.459999084472656 +wendy van buren 92.81999969482422 +wendy white 73.68000030517578 +wendy xylophone 76.69999694824219 +wendy xylophone 90.60000038146973 +wendy young 8.449999809265137 +wendy young 33.7599983215332 +xavier allen 45.68000030517578 +xavier allen 83.93000030517578 +xavier allen 98.22000122070312 +xavier brown 7.789999961853027 +xavier brown 90.7300033569336 +xavier brown 96.2300033569336 +xavier carson 20.790000915527344 +xavier carson 94.68000030517578 +xavier davidson 15.920000076293945 +xavier davidson 82.41000366210938 +xavier davidson 106.5199966430664 +xavier ellison 12.850000381469727 +xavier ellison 77.97000122070312 +xavier garcia 70.04000091552734 +xavier hernandez 6.670000076293945 +xavier hernandez 38.56999969482422 +xavier hernandez 67.26000213623047 +xavier ichabod 4.71999979019165 +xavier ichabod 71.19000244140625 +xavier johnson 27.299999237060547 +xavier johnson 203.65999794006348 +xavier king 8.569999694824219 +xavier king 87.22000122070312 +xavier laertes 15.899999618530273 +xavier ovid 112.91000366210938 +xavier polk 13.869999885559082 +xavier polk 61.209999084472656 +xavier polk 72.62000274658203 +xavier polk 76.93000030517578 +xavier quirinius 62.52000045776367 +xavier quirinius 83.01000022888184 +xavier quirinius 89.55000305175781 +xavier quirinius 97.14999961853027 +xavier thompson 9.930000305175781 +xavier underhill 47.27000045776367 +xavier white 59.20000171661377 +xavier white 75.29000091552734 +xavier xylophone 79.41999816894531 +xavier zipper 8.449999809265137 +yuri allen 52.849998474121094 +yuri allen 94.98999977111816 +yuri brown 75.19000244140625 +yuri brown 84.02999877929688 +yuri carson 6.289999961853027 +yuri carson 91.16000366210938 +yuri ellison 1.1200000047683716 +yuri ellison 98.82999801635742 +yuri falkner 39.6299991607666 +yuri falkner 86.0 +yuri garcia 27.65999984741211 +yuri hernandez 2.069999933242798 +yuri johnson 0.12999999523162842 +yuri johnson 39.900001525878906 +yuri johnson 48.220001220703125 +yuri king 69.59000015258789 +yuri laertes 37.59000015258789 +yuri laertes 61.95000076293945 +yuri nixon 2.200000047683716 +yuri nixon 82.81000328063965 +yuri polk 26.760000228881836 +yuri polk 28.790000915527344 +yuri polk 105.11999702453613 +yuri quirinius 10.260000228881836 +yuri quirinius 54.310001373291016 +yuri quirinius 57.93000030517578 +yuri steinbeck 17.790000915527344 +yuri steinbeck 75.87999725341797 +yuri thompson 14.920000076293945 +yuri underhill 23.770000457763672 +yuri underhill 83.87000274658203 +yuri white 34.58000183105469 +yuri xylophone 20.3799991607666 +zach allen 65.43000030517578 +zach brown 48.0099983215332 +zach brown 49.119998931884766 +zach brown 57.08000183105469 +zach brown 67.37999725341797 +zach brown 100.46000289916992 +zach carson 95.86999893188477 +zach ellison 6.840000152587891 +zach falkner 9.130000114440918 +zach falkner 91.41999816894531 +zach garcia 32.20000076293945 +zach garcia 84.37999725341797 +zach garcia 106.86999893188477 +zach garcia 167.62000274658203 +zach ichabod 64.25 +zach ichabod 106.69000244140625 +zach king 46.18000030517578 +zach king 70.51000213623047 +zach king 86.93000030517578 +zach miller 2.5999999046325684 +zach miller 21.280000686645508 +zach miller 53.27000045776367 +zach ovid 0.10000000149011612 +zach ovid 23.06999969482422 +zach ovid 92.55000305175781 +zach ovid 94.33999633789062 +zach quirinius 39.209999084472656 +zach robinson 122.81000137329102 +zach steinbeck 85.48999786376953 +zach steinbeck 90.05000305175781 +zach thompson 71.5 +zach thompson 91.63999938964844 +zach underhill 86.22000122070312 +zach white 70.52999877929688 +zach xylophone 43.84999942779541 +zach xylophone 71.01000213623047 +zach young 71.31999969482422 +zach zipper 52.60000133514404 +zach zipper 85.87000274658203 +zach zipper 94.43000030517578 diff --git a/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-2-5f0eab306ea3c22b11ace9b542a7ee56 b/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-2-5f0eab306ea3c22b11ace9b542a7ee56 new file mode 100644 index 0000000000000..e55bede9242e5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-2-5f0eab306ea3c22b11ace9b542a7ee56 @@ -0,0 +1,1049 @@ + 257.04000091552734 + 261.16000175476074 + 284.2699966430664 +alice allen 73.62999725341797 +alice allen 195.0099983215332 +alice allen 196.729998588562 +alice brown 245.52000427246094 +alice carson 424.63000106811523 +alice davidson 319.00999450683594 +alice falkner 90.25 +alice garcia 174.36999893188477 +alice hernandez 185.6699981689453 +alice hernandez 380.1699981689453 +alice johnson 367.2900071144104 +alice king 58.78999900817871 +alice king 294.1199951171875 +alice king 371.23999404907227 +alice laertes 68.94999694824219 +alice laertes 258.3999938964844 +alice miller 154.19000244140625 +alice nixon 209.91000366210938 +alice nixon 246.36000442504883 +alice nixon 260.19000244140625 +alice ovid 49.8199987411499 +alice polk 148.63999938964844 +alice quirinius 239.81999588012695 +alice quirinius 301.4100036621094 +alice robinson 140.47999572753906 +alice robinson 266.4900016784668 +alice steinbeck 169.76000213623047 +alice steinbeck 186.70999908447266 +alice steinbeck 446.8099937438965 +alice underhill 98.18000030517578 +alice van buren 112.42000198364258 +alice xylophone 78.20999908447266 +alice xylophone 91.22000122070312 +alice xylophone 413.1199951171875 +alice zipper 89.93000030517578 +alice zipper 279.54000091552734 +alice zipper 293.25000381469727 +bob brown 188.89999389648438 +bob brown 228.80999946594238 +bob brown 247.37999725341797 +bob carson 207.67000198364258 +bob davidson 53.93000030517578 +bob davidson 113.83999919891357 +bob davidson 259.0899963378906 +bob ellison 65.0199966430664 +bob ellison 80.30000305175781 +bob ellison 243.86000061035156 +bob ellison 245.02999877929688 +bob falkner 208.82000160217285 +bob garcia 33.410000801086426 +bob garcia 87.56999969482422 +bob garcia 120.17999649047852 +bob garcia 148.65999841690063 +bob garcia 178.87000274658203 +bob hernandez 337.23999977111816 +bob ichabod 82.55999755859375 +bob king 114.11000156402588 +bob king 134.81999588012695 +bob king 152.7699956893921 +bob laertes 42.89999961853027 +bob laertes 393.99999433755875 +bob miller 146.1500015258789 +bob ovid 62.849998474121094 +bob ovid 88.77999877929688 +bob ovid 97.08999633789062 +bob ovid 102.93000030517578 +bob polk 261.4599976539612 +bob quirinius 298.7199897766113 +bob steinbeck 103.01999950408936 +bob van buren 174.89999771118164 +bob white 194.25 +bob white 347.7799949645996 +bob xylophone 19.690000534057617 +bob xylophone 191.52999687194824 +bob young 78.17999649047852 +bob zipper 132.86000061035156 +bob zipper 139.6900042295456 +bob zipper 295.59000039100647 +calvin allen 255.68000411987305 +calvin brown 85.9000015258789 +calvin brown 238.02000427246094 +calvin brown 275.8699951171875 +calvin carson 80.2300033569336 +calvin davidson 31.770000457763672 +calvin davidson 181.76000213623047 +calvin ellison 188.0300006866455 +calvin falkner 93.61000061035156 +calvin falkner 94.30999755859375 +calvin falkner 125.91999816894531 +calvin falkner 137.1699981689453 +calvin falkner 140.99999594688416 +calvin falkner 168.81999969482422 +calvin garcia 307.439998626709 +calvin hernandez 303.4599952697754 +calvin johnson 152.8300018310547 +calvin laertes 150.69999885559082 +calvin laertes 216.81000137329102 +calvin nixon 131.57999801635742 +calvin nixon 143.3699951171875 +calvin nixon 196.34000301361084 +calvin ovid 69.95999908447266 +calvin ovid 176.13999938964844 +calvin ovid 176.3800048828125 +calvin ovid 248.65999603271484 +calvin polk 147.04000091552734 +calvin quirinius 226.66999435424805 +calvin quirinius 266.7100009918213 +calvin robinson 289.7900047302246 +calvin steinbeck 92.05000305175781 +calvin steinbeck 118.15000057220459 +calvin steinbeck 333.6000003814697 +calvin thompson 93.7300033569336 +calvin thompson 249.56000137329102 +calvin underhill 208.3400001525879 +calvin van buren 136.51000213623047 +calvin van buren 347.0999946594238 +calvin white 90.69000244140625 +calvin white 112.15999984741211 +calvin xylophone 25.420000076293945 +calvin xylophone 237.71999740600586 +calvin xylophone 315.2099952697754 +calvin young 222.96000289916992 +calvin young 243.3199977874756 +calvin zipper 95.37999725341797 +calvin zipper 531.3600015640259 +david allen 202.43000030517578 +david allen 302.4399948120117 +david brown 93.63999938964844 +david brown 258.05999851226807 +david davidson 74.1500015258789 +david davidson 95.80999755859375 +david davidson 106.50000202655792 +david davidson 149.94000244140625 +david ellison 85.2300033569336 +david ellison 94.1500015258789 +david ellison 208.3900032043457 +david hernandez 99.91000366210938 +david ichabod 82.55000305175781 +david ichabod 320.47999143600464 +david laertes 250.1699981689453 +david nixon 174.58999633789062 +david ovid 198.21000289916992 +david ovid 230.47999954223633 +david quirinius 29.239999771118164 +david quirinius 79.97000122070312 +david quirinius 180.92999649047852 +david robinson 147.65999603271484 +david robinson 168.7100009918213 +david thompson 41.88999938964844 +david underhill 97.55999755859375 +david underhill 277.5999984741211 +david underhill 369.4600009918213 +david van buren 83.56999969482422 +david van buren 289.189998626709 +david white 124.6099967956543 +david xylophone 135.70000076293945 +david xylophone 237.06000137329102 +david xylophone 338.20999908447266 +david young 172.49000549316406 +david young 184.9800033569336 +ethan allen 240.42000198364258 +ethan brown 61.86000061035156 +ethan brown 73.18000030517578 +ethan brown 105.29000043869019 +ethan brown 177.8300018310547 +ethan brown 185.98999691009521 +ethan brown 284.729998588562 +ethan carson 265.22999572753906 +ethan ellison 166.5 +ethan ellison 244.99000671505928 +ethan falkner 59.43000030517578 +ethan falkner 196.17000198364258 +ethan garcia 271.5999946594238 +ethan hernandez 264.50999450683594 +ethan johnson 90.05000305175781 +ethan king 36.49000024795532 +ethan laertes 95.06999969482422 +ethan laertes 96.29000091552734 +ethan laertes 189.66000270843506 +ethan laertes 192.79999923706055 +ethan laertes 249.04000091552734 +ethan laertes 249.76000213623047 +ethan laertes 369.9599952697754 +ethan miller 314.5599994659424 +ethan nixon 493.03000259399414 +ethan ovid 57.290000915527344 +ethan polk 2.3499999046325684 +ethan polk 59.869998931884766 +ethan polk 219.6599998474121 +ethan polk 263.8600025177002 +ethan quirinius 97.23999786376953 +ethan quirinius 111.70999908447266 +ethan quirinius 317.69000363349915 +ethan robinson 78.62000274658203 +ethan robinson 149.5800018310547 +ethan underhill 231.25000381469727 +ethan van buren 152.60000228881836 +ethan white 155.81999969482422 +ethan white 235.55999755859375 +ethan xylophone 414.61000061035156 +ethan zipper 97.51000213623047 +ethan zipper 297.20999908447266 +fred davidson 78.30999755859375 +fred davidson 105.8499984741211 +fred davidson 220.56000137329102 +fred ellison 56.489999771118164 +fred ellison 96.77999877929688 +fred ellison 199.52000045776367 +fred falkner 66.77999973297119 +fred falkner 85.0 +fred falkner 169.91000366210938 +fred hernandez 117.85000228881836 +fred ichabod 81.31999969482422 +fred ichabod 202.45000457763672 +fred johnson 96.08999633789062 +fred king 140.24999618530273 +fred king 343.82000064849854 +fred laertes 57.63999938964844 +fred miller 176.18000030517578 +fred nixon 28.690000534057617 +fred nixon 187.40999603271484 +fred nixon 246.77000045776367 +fred nixon 338.34999084472656 +fred polk 90.12000274658203 +fred polk 323.1899948120117 +fred polk 357.19000244140625 +fred polk 496.16999435424805 +fred quirinius 218.82999801635742 +fred quirinius 224.12000179290771 +fred robinson 89.02999877929688 +fred steinbeck 32.22999954223633 +fred steinbeck 91.05000305175781 +fred steinbeck 231.92000198364258 +fred underhill 183.31999969482422 +fred van buren 83.58000183105469 +fred van buren 318.38000106811523 +fred van buren 346.1400008201599 +fred van buren 391.9999942779541 +fred white 187.38000106811523 +fred young 97.70999908447266 +fred young 141.22999954223633 +fred zipper 163.89999771118164 +gabriella allen 274.8599967956543 +gabriella allen 283.5 +gabriella brown 163.89999961853027 +gabriella brown 465.0 +gabriella carson 147.78999710083008 +gabriella davidson 263.64000415802 +gabriella ellison 71.54000091552734 +gabriella ellison 188.55999755859375 +gabriella falkner 51.720001220703125 +gabriella falkner 87.61000061035156 +gabriella falkner 162.21999835968018 +gabriella garcia 43.0099983215332 +gabriella hernandez 190.5500030517578 +gabriella hernandez 267.4700012207031 +gabriella ichabod 71.12999725341797 +gabriella ichabod 90.3499984741211 +gabriella ichabod 148.6999969482422 +gabriella ichabod 175.70000076293945 +gabriella ichabod 285.72999572753906 +gabriella king 166.75000190734863 +gabriella king 177.6999969482422 +gabriella laertes 65.37999725341797 +gabriella miller 148.4800033569336 +gabriella ovid 92.4000015258789 +gabriella ovid 137.82999801635742 +gabriella polk 244.07000350952148 +gabriella polk 282.00000762939453 +gabriella steinbeck 272.2799987792969 +gabriella steinbeck 461.060001373291 +gabriella thompson 88.36000061035156 +gabriella thompson 94.25 +gabriella thompson 158.80999755859375 +gabriella van buren 146.0800018310547 +gabriella van buren 151.63999938964844 +gabriella white 138.72000122070312 +gabriella young 30.739999771118164 +gabriella young 146.62999725341797 +gabriella zipper 91.62999725341797 +gabriella zipper 357.5099983215332 +holly allen 44.56999969482422 +holly brown 173.64999389648438 +holly brown 174.2100067138672 +holly falkner 166.22999572753906 +holly hernandez 180.0800018310547 +holly hernandez 248.71000480651855 +holly hernandez 336.87000274658203 +holly hernandez 523.2800025939941 +holly ichabod 179.37000274658203 +holly ichabod 180.18000030517578 +holly ichabod 184.66000366210938 +holly johnson 64.36000061035156 +holly johnson 145.61000061035156 +holly johnson 157.12999725341797 +holly king 275.2299995422363 +holly king 288.52000427246094 +holly laertes 246.21000289916992 +holly miller 290.21999740600586 +holly nixon 177.39999389648438 +holly nixon 228.58999633789062 +holly polk 98.30999755859375 +holly polk 307.0799951553345 +holly robinson 219.27999877929688 +holly thompson 75.41999816894531 +holly thompson 86.69000244140625 +holly thompson 523.360002592206 +holly underhill 96.68000030517578 +holly underhill 163.54000091552734 +holly underhill 187.47000122070312 +holly underhill 328.0099983215332 +holly van buren 161.7699966430664 +holly white 122.98999786376953 +holly white 335.93999576568604 +holly xylophone 191.34000396728516 +holly young 60.220001220703125 +holly young 297.20999908447266 +holly zipper 99.12999725341797 +holly zipper 99.29000091552734 +irene allen 234.6400032043457 +irene brown 4.789999961853027 +irene brown 176.4499969482422 +irene brown 338.2099952697754 +irene carson 292.0 +irene ellison 201.06000137329102 +irene ellison 230.79000091552734 +irene falkner 99.91999816894531 +irene falkner 210.11000061035156 +irene garcia 40.78999996185303 +irene garcia 86.93000030517578 +irene garcia 183.02000045776367 +irene ichabod 99.62000274658203 +irene ichabod 281.96999740600586 +irene johnson 243.59999752044678 +irene laertes 112.54000091552734 +irene laertes 227.45000076293945 +irene laertes 246.53000259399414 +irene miller 395.9100036621094 +irene nixon 29.780000686645508 +irene nixon 199.45999908447266 +irene nixon 261.46000480651855 +irene ovid 158.97000122070312 +irene ovid 339.94000244140625 +irene ovid 362.82000732421875 +irene polk 95.83999633789062 +irene polk 183.43000411987305 +irene polk 258.7100033760071 +irene polk 284.6300048828125 +irene polk 507.2400016784668 +irene quirinius 157.5800018310547 +irene quirinius 250.61000061035156 +irene quirinius 431.6499996185303 +irene robinson 191.72999572753906 +irene steinbeck 94.33000183105469 +irene thompson 256.0 +irene underhill 135.55999755859375 +irene underhill 327.0299892425537 +irene van buren 54.439998626708984 +irene van buren 193.71000289916992 +irene xylophone 168.5 +jessica brown 422.5299949645996 +jessica carson 103.66000366210938 +jessica carson 144.92000198364258 +jessica carson 259.1099967956543 +jessica davidson 95.33999633789062 +jessica davidson 99.20999908447266 +jessica davidson 137.17000198364258 +jessica davidson 227.79999923706055 +jessica ellison 207.35000228881836 +jessica ellison 237.4300022125244 +jessica falkner 99.6500015258789 +jessica garcia 174.70999908447266 +jessica garcia 185.62000179290771 +jessica ichabod 124.59000015258789 +jessica johnson 272.0500030517578 +jessica johnson 294.2899990081787 +jessica miller 77.83999633789062 +jessica nixon 77.0999984741211 +jessica nixon 90.06999969482422 +jessica ovid 71.68000030517578 +jessica ovid 309.44000244140625 +jessica polk 472.2099952697754 +jessica quirinius 35.619998931884766 +jessica quirinius 192.7000026702881 +jessica quirinius 208.6500015258789 +jessica quirinius 370.0599937438965 +jessica robinson 254.5300064086914 +jessica thompson 115.9000015258789 +jessica thompson 180.60000610351562 +jessica underhill 199.10999870300293 +jessica underhill 234.29000091552734 +jessica underhill 257.09000396728516 +jessica van buren 9.739999771118164 +jessica white 96.62000274658203 +jessica white 166.54000091552734 +jessica white 240.52999877929688 +jessica white 432.17999362945557 +jessica white 497.6400032043457 +jessica xylophone 385.4799995422363 +jessica young 47.410000801086426 +jessica young 240.6500015258789 +jessica zipper 323.4199962615967 +jessica zipper 344.8399953842163 +jessica zipper 480.06999588012695 +katie allen 312.9700012207031 +katie brown 573.4599933624268 +katie davidson 96.91000366210938 +katie ellison 163.52999877929688 +katie ellison 384.4699947834015 +katie falkner 125.57000160217285 +katie garcia 84.4000015258789 +katie garcia 160.28999710083008 +katie hernandez 257.9600028991699 +katie ichabod 187.63999557495117 +katie ichabod 274.97999572753906 +katie ichabod 362.9200019836426 +katie king 97.80999755859375 +katie king 169.56999969482422 +katie king 314.1999969482422 +katie miller 31.399999618530273 +katie miller 228.40999603271484 +katie nixon 23.190000534057617 +katie ovid 207.1200065612793 +katie polk 143.2599983215332 +katie polk 247.02000045776367 +katie robinson 83.84999942779541 +katie van buren 297.0300064086914 +katie van buren 464.54999351501465 +katie white 344.1700019836426 +katie white 465.8599934577942 +katie xylophone 175.89999675750732 +katie young 31.010000228881836 +katie young 72.51000213623047 +katie young 97.56999969482422 +katie zipper 101.9000015258789 +katie zipper 314.75 +luke allen 89.55000305175781 +luke allen 133.4800033569336 +luke allen 210.8800048828125 +luke allen 392.0300064086914 +luke allen 420.6299934387207 +luke brown 129.20999908447266 +luke davidson 28.950000762939453 +luke davidson 106.41000080108643 +luke ellison 42.09000027179718 +luke ellison 136.52000427246094 +luke ellison 187.51000213623047 +luke falkner 172.8799991607666 +luke falkner 216.0199966430664 +luke garcia 50.94000053405762 +luke garcia 345.1200008392334 +luke ichabod 67.90000057220459 +luke ichabod 97.87000274658203 +luke johnson 59.00999927520752 +luke johnson 105.32000160217285 +luke johnson 187.2899990081787 +luke laertes 105.42000198364258 +luke laertes 147.14999723434448 +luke laertes 158.86000061035156 +luke laertes 167.01999855041504 +luke laertes 281.19999504089355 +luke miller 97.6500015258789 +luke ovid 186.53000259399414 +luke ovid 340.1300048828125 +luke polk 95.27999877929688 +luke polk 277.6700019836426 +luke quirinius 115.83999633789062 +luke robinson 137.33999633789062 +luke robinson 145.23999786376953 +luke thompson 94.37999725341797 +luke underhill 96.94000244140625 +luke underhill 194.73999786376953 +luke underhill 372.6899948120117 +luke van buren 193.93999862670898 +luke white 67.12000274658203 +luke xylophone 102.37999725341797 +luke zipper 223.54000282287598 +mike allen 79.60999870300293 +mike brown 202.81999588012695 +mike carson 81.66000366210938 +mike carson 105.02999877929688 +mike carson 405.4499931335449 +mike davidson 137.74999809265137 +mike davidson 393.17999267578125 +mike ellison 79.37999725341797 +mike ellison 85.73999786376953 +mike ellison 127.15999603271484 +mike ellison 228.07999992370605 +mike ellison 263.8899955749512 +mike falkner 254.50000381469727 +mike garcia 70.8499984741211 +mike garcia 173.63999938964844 +mike garcia 177.5199966430664 +mike hernandez 59.45000076293945 +mike hernandez 327.6900062561035 +mike ichabod 64.7699966430664 +mike king 78.26000213623047 +mike king 84.2300033569336 +mike king 94.68000030517578 +mike king 133.5900001525879 +mike king 134.87999725341797 +mike king 173.45999908447266 +mike miller 57.890000343322754 +mike nixon 92.95999908447266 +mike nixon 203.68999862670898 +mike polk 32.140000343322754 +mike polk 99.68000030517578 +mike polk 306.61000061035156 +mike quirinius 89.37999725341797 +mike steinbeck 85.13999938964844 +mike steinbeck 97.45999908447266 +mike steinbeck 153.86000204086304 +mike steinbeck 221.21999502182007 +mike van buren 80.83999633789062 +mike van buren 174.21000289916992 +mike white 91.87999725341797 +mike white 341.80999755859375 +mike white 341.86000061035156 +mike white 389.20000076293945 +mike young 74.58999633789062 +mike young 83.54000091552734 +mike young 112.19000101089478 +mike zipper 86.98999786376953 +mike zipper 97.38999938964844 +mike zipper 174.61000061035156 +nick allen 173.32000207901 +nick allen 257.7300033569336 +nick brown 192.45000076293945 +nick davidson 258.9799919128418 +nick ellison 183.34000396728516 +nick ellison 193.02000427246094 +nick falkner 10.130000114440918 +nick falkner 182.72000122070312 +nick garcia 142.65999603271484 +nick garcia 183.7699966430664 +nick garcia 277.8299951553345 +nick ichabod 110.43999862670898 +nick ichabod 112.54999923706055 +nick ichabod 241.68999481201172 +nick johnson 192.56000518798828 +nick johnson 325.9499976634979 +nick laertes 96.25 +nick miller 82.97000122070312 +nick nixon 96.37999725341797 +nick ovid 184.3699951171875 +nick polk 199.57000064849854 +nick quirinius 174.80999755859375 +nick quirinius 243.8300018310547 +nick robinson 129.65999603271484 +nick robinson 216.54999923706055 +nick steinbeck 97.83000183105469 +nick thompson 205.4900016784668 +nick underhill 166.42000007629395 +nick van buren 222.6500015258789 +nick xylophone 75.3499984741211 +nick young 332.23999786376953 +nick young 346.41000083088875 +nick zipper 222.9199981689453 +nick zipper 529.7199974060059 +oscar allen 246.42999839782715 +oscar brown 274.01000022888184 +oscar carson 78.9800033569336 +oscar carson 87.4800033569336 +oscar carson 98.51000213623047 +oscar carson 203.86000442504883 +oscar carson 321.82000064849854 +oscar davidson 361.6699981689453 +oscar ellison 146.44000244140625 +oscar ellison 234.32000160217285 +oscar falkner 98.4800033569336 +oscar garcia 231.04000091552734 +oscar hernandez 85.48999786376953 +oscar hernandez 95.4800033569336 +oscar ichabod 71.80000305175781 +oscar ichabod 123.78000068664551 +oscar ichabod 173.31000518798828 +oscar ichabod 251.22000122070312 +oscar johnson 146.27000427246094 +oscar johnson 260.1600036621094 +oscar king 124.2699966430664 +oscar king 249.5399990081787 +oscar king 284.8599910736084 +oscar laertes 15.640000343322754 +oscar laertes 254.8499984741211 +oscar laertes 261.41000175476074 +oscar laertes 261.8400020599365 +oscar nixon 41.619998931884766 +oscar ovid 82.23999786376953 +oscar ovid 187.76000213623047 +oscar ovid 260.6100044250488 +oscar polk 63.900001525878906 +oscar polk 252.71000289916992 +oscar quirinius 73.4800033569336 +oscar quirinius 165.3800048828125 +oscar quirinius 244.2699966430664 +oscar quirinius 248.75 +oscar robinson 93.31999969482422 +oscar robinson 163.55999755859375 +oscar robinson 191.8300018310547 +oscar robinson 315.1999912261963 +oscar steinbeck 376.6899948120117 +oscar thompson 131.1400032043457 +oscar thompson 148.01000213623047 +oscar thompson 325.42000579833984 +oscar thompson 545.7399940490723 +oscar underhill 87.4000015258789 +oscar van buren 61.880001068115234 +oscar van buren 188.8699951171875 +oscar van buren 209.53000235557556 +oscar white 129.73999786376953 +oscar white 148.9800033569336 +oscar white 275.1500015258789 +oscar white 303.8599910736084 +oscar xylophone 115.22999954223633 +oscar xylophone 319.75000381469727 +oscar xylophone 475.3300018310547 +oscar zipper 109.53999710083008 +oscar zipper 214.40999603271484 +oscar zipper 214.6500015258789 +priscilla brown 77.56999969482422 +priscilla brown 165.5199966430664 +priscilla brown 408.4499969482422 +priscilla carson 168.8300018310547 +priscilla carson 195.7900047302246 +priscilla carson 207.5300006866455 +priscilla ichabod 92.61000061035156 +priscilla ichabod 206.16000366210938 +priscilla johnson 89.1500015258789 +priscilla johnson 156.4600067138672 +priscilla johnson 158.88000106811523 +priscilla johnson 190.61000061035156 +priscilla johnson 211.01000022888184 +priscilla king 371.9299964904785 +priscilla nixon 95.80999755859375 +priscilla nixon 278.87999725341797 +priscilla ovid 96.27000284194946 +priscilla ovid 198.3400001525879 +priscilla polk 252.5800018310547 +priscilla quirinius 131.8499994277954 +priscilla thompson 230.36000156402588 +priscilla underhill 143.56999969482422 +priscilla underhill 354.37000274658203 +priscilla van buren 82.72000122070312 +priscilla van buren 145.61000061035156 +priscilla van buren 183.72000122070312 +priscilla white 78.27999877929688 +priscilla xylophone 21.489999771118164 +priscilla xylophone 159.26000213623047 +priscilla xylophone 406.1000007688999 +priscilla young 163.2900013923645 +priscilla young 260.59000366926193 +priscilla zipper 311.399995803833 +priscilla zipper 327.97999572753906 +quinn allen 257.94000244140625 +quinn allen 365.2299995422363 +quinn brown 80.58000183105469 +quinn brown 80.81999969482422 +quinn brown 198.71000289916992 +quinn davidson 83.4000015258789 +quinn davidson 95.11000061035156 +quinn davidson 154.79000091552734 +quinn davidson 227.13999938964844 +quinn ellison 237.17000007629395 +quinn ellison 361.14000129699707 +quinn garcia 92.33000183105469 +quinn garcia 148.63999938964844 +quinn garcia 226.78999710083008 +quinn garcia 246.70000076293945 +quinn ichabod 89.63999938964844 +quinn king 74.62000274658203 +quinn king 86.2300033569336 +quinn laertes 112.36000061035156 +quinn laertes 243.6900019645691 +quinn laertes 265.51000213623047 +quinn nixon 149.3300018310547 +quinn ovid 393.2099943161011 +quinn quirinius 266.8200035095215 +quinn robinson 247.6400032043457 +quinn steinbeck 144.81999969482422 +quinn steinbeck 213.65999507904053 +quinn thompson 156.51000213623047 +quinn thompson 274.1599931716919 +quinn underhill 248.3000030517578 +quinn underhill 252.61999130249023 +quinn underhill 321.9799976348877 +quinn van buren 82.5199966430664 +quinn young 90.97999954223633 +quinn zipper 58.0 +quinn zipper 249.38999938964844 +rachel allen 15.8100004196167 +rachel allen 151.80999755859375 +rachel brown 193.5800018310547 +rachel brown 312.0800018310547 +rachel brown 347.7999954223633 +rachel brown 423.98999214172363 +rachel brown 437.64999771118164 +rachel carson 98.95999908447266 +rachel carson 385.3799934387207 +rachel davidson 396.38999938964844 +rachel ellison 299.12000465393066 +rachel falkner 88.80000305175781 +rachel falkner 99.23999786376953 +rachel falkner 172.54999542236328 +rachel falkner 233.55999755859375 +rachel johnson 197.92999649047852 +rachel king 36.220001220703125 +rachel king 219.8400001525879 +rachel laertes 97.17000198364258 +rachel laertes 109.5999984741211 +rachel ovid 80.20999872684479 +rachel ovid 260.18999683856964 +rachel polk 89.27999877929688 +rachel quirinius 205.1400022506714 +rachel robinson 254.1300015449524 +rachel robinson 286.0400047302246 +rachel robinson 332.4199981689453 +rachel thompson 137.73000198602676 +rachel thompson 213.31000137329102 +rachel thompson 380.85999488830566 +rachel underhill 175.6099967956543 +rachel white 94.72000122070312 +rachel white 196.18000030517578 +rachel young 230.6400032043457 +rachel zipper 148.9000015258789 +rachel zipper 238.98000192642212 +sarah carson 175.62000274658203 +sarah carson 307.70000088214874 +sarah carson 386.8999948501587 +sarah ellison 161.80999946594238 +sarah falkner 99.36000061035156 +sarah falkner 281.62000274658203 +sarah garcia 73.6500015258789 +sarah garcia 153.73000144958496 +sarah garcia 312.8899955749512 +sarah ichabod 81.31999969482422 +sarah ichabod 97.26000213623047 +sarah johnson 140.37999725341797 +sarah johnson 177.57000732421875 +sarah johnson 248.4499969482422 +sarah johnson 309.1800022125244 +sarah king 216.75 +sarah king 268.5399932861328 +sarah miller 222.31000518798828 +sarah ovid 146.25000381469727 +sarah robinson 143.43000030517578 +sarah robinson 310.75 +sarah steinbeck 208.72000122070312 +sarah white 140.22999572753906 +sarah white 181.86000061035156 +sarah xylophone 68.31999969482422 +sarah young 185.80999755859375 +sarah zipper 168.22000122070312 +tom brown 181.1000051498413 +tom brown 404.3500061035156 +tom carson 142.60999822616577 +tom carson 299.57999992370605 +tom carson 592.3499927520752 +tom davidson 180.61000061035156 +tom ellison 98.2300033569336 +tom ellison 154.58999633789062 +tom ellison 173.02999877929688 +tom falkner 88.22000122070312 +tom falkner 139.11000442504883 +tom hernandez 81.63999938964844 +tom hernandez 263.67000579833984 +tom ichabod 214.0699977874756 +tom johnson 405.95000076293945 +tom johnson 438.9099922180176 +tom king 218.18000030517578 +tom laertes 244.37000274658203 +tom laertes 473.0999984741211 +tom miller 68.25 +tom miller 85.59000015258789 +tom miller 127.56999969482422 +tom nixon 85.02999877929688 +tom ovid 217.32000160217285 +tom polk 188.87000274658203 +tom polk 206.52000045776367 +tom quirinius 120.27000427246094 +tom quirinius 232.63000202178955 +tom robinson 90.69000244140625 +tom robinson 98.72000122070312 +tom robinson 99.1500015258789 +tom robinson 209.5399932861328 +tom steinbeck 277.7100009918213 +tom van buren 40.779998779296875 +tom van buren 217.70000076293945 +tom van buren 375.2099964618683 +tom white 223.4700050354004 +tom young 174.36000061035156 +tom young 304.8199977874756 +tom zipper 213.7900047302246 +ulysses brown 247.1500015258789 +ulysses carson 77.41999816894531 +ulysses carson 79.54000091552734 +ulysses carson 150.93000030517578 +ulysses carson 162.24000549316406 +ulysses davidson 414.7100009918213 +ulysses ellison 96.7300033569336 +ulysses garcia 89.80000305175781 +ulysses hernandez 106.29999542236328 +ulysses hernandez 134.44000244140625 +ulysses hernandez 160.22000122070312 +ulysses ichabod 98.56999969482422 +ulysses ichabod 309.34999656677246 +ulysses johnson 152.47000122070312 +ulysses king 244.7100067138672 +ulysses laertes 138.4400042295456 +ulysses laertes 173.55999755859375 +ulysses laertes 256.91999912261963 +ulysses miller 76.27999877929688 +ulysses miller 417.67000102996826 +ulysses nixon 174.56999969482422 +ulysses ovid 130.13000106811523 +ulysses polk 123.9399995803833 +ulysses polk 149.95999908447266 +ulysses polk 205.2400016784668 +ulysses polk 237.5699920654297 +ulysses quirinius 330.4700012207031 +ulysses robinson 79.48999786376953 +ulysses steinbeck 144.8300018310547 +ulysses steinbeck 155.66000366210938 +ulysses thompson 159.92000579833984 +ulysses underhill 81.58000183105469 +ulysses underhill 88.4800033569336 +ulysses underhill 99.66999816894531 +ulysses underhill 135.55999755859375 +ulysses underhill 189.1099977493286 +ulysses underhill 289.6800003051758 +ulysses underhill 385.60000228881836 +ulysses van buren 95.52999877929688 +ulysses white 188.8300018310547 +ulysses white 305.79000091552734 +ulysses xylophone 54.099998474121094 +ulysses xylophone 205.2099952697754 +ulysses xylophone 251.94000148773193 +ulysses young 100.77000045776367 +ulysses young 275.8300018310547 +ulysses young 522.1700019836426 +victor allen 220.1699981689453 +victor allen 222.10000228881836 +victor brown 77.88999938964844 +victor brown 90.37999725341797 +victor brown 91.97000122070312 +victor brown 455.25000381469727 +victor davidson 149.06000137329102 +victor davidson 291.48000717163086 +victor davidson 321.25 +victor ellison 314.37000274658203 +victor ellison 442.50000190734863 +victor hernandez 69.87999725341797 +victor hernandez 99.85000038146973 +victor hernandez 143.02000045776367 +victor hernandez 160.38999938964844 +victor hernandez 391.2999954223633 +victor johnson 145.18000030517578 +victor johnson 190.27000045776367 +victor johnson 308.1900006532669 +victor king 108.10000228881836 +victor king 310.5 +victor laertes 145.42999267578125 +victor laertes 214.72999572753906 +victor miller 173.76000022888184 +victor nixon 68.5 +victor nixon 269.5899963378906 +victor ovid 151.39999771118164 +victor polk 175.8799991607666 +victor quirinius 65.55000305175781 +victor quirinius 168.5500030517578 +victor robinson 177.9100022315979 +victor robinson 204.09999084472656 +victor steinbeck 52.720001220703125 +victor steinbeck 220.45999908447266 +victor steinbeck 309.4900064468384 +victor thompson 58.65999984741211 +victor van buren 206.77999877929688 +victor van buren 222.44000244140625 +victor white 156.36999893188477 +victor white 167.2699966430664 +victor xylophone 158.36999893188477 +victor xylophone 161.54000091552734 +victor xylophone 234.76000308990479 +victor xylophone 267.82999420166016 +victor xylophone 314.95000076293945 +victor young 88.55000305175781 +victor zipper 192.92999649047852 +wendy allen 56.06999969482422 +wendy allen 66.16000306606293 +wendy allen 267.3199996948242 +wendy brown 453.53000259399414 +wendy brown 525.5100040435791 +wendy ellison 193.95000457763672 +wendy ellison 260.9099998474121 +wendy falkner 77.36000061035156 +wendy falkner 97.68000030517578 +wendy falkner 128.30999565124512 +wendy garcia 4.409999847412109 +wendy garcia 76.72000122070312 +wendy garcia 189.42999839782715 +wendy garcia 265.5900001525879 +wendy hernandez 48.11000061035156 +wendy ichabod 104.3700008392334 +wendy king 156.89999771118164 +wendy king 183.31999969482422 +wendy king 403.27000427246094 +wendy laertes 79.98999786376953 +wendy laertes 165.0999984741211 +wendy laertes 365.0 +wendy miller 72.9500002861023 +wendy miller 313.8300037384033 +wendy nixon 45.91999816894531 +wendy nixon 60.2599983215332 +wendy ovid 95.33000183105469 +wendy ovid 180.36000061035156 +wendy polk 386.7400016784668 +wendy polk 443.3400018811226 +wendy quirinius 152.04999828338623 +wendy quirinius 240.23999977111816 +wendy robinson 71.06999969482422 +wendy robinson 249.35000610351562 +wendy robinson 391.4699993133545 +wendy steinbeck 92.11000061035156 +wendy thompson 136.35000228881836 +wendy thompson 183.1500015258789 +wendy underhill 318.6500015258789 +wendy underhill 320.75000190734863 +wendy underhill 328.2300033569336 +wendy van buren 57.459999084472656 +wendy van buren 92.81999969482422 +wendy white 171.36000061035156 +wendy xylophone 153.62999725341797 +wendy xylophone 223.94999885559082 +wendy young 40.22000026702881 +wendy young 513.8299942016602 +xavier allen 102.97000122070312 +xavier allen 168.3300018310547 +xavier allen 197.45999908447266 +xavier brown 55.20000076293945 +xavier brown 90.7300033569336 +xavier brown 96.2300033569336 +xavier carson 193.63999938964844 +xavier carson 265.1600036621094 +xavier davidson 63.349998474121094 +xavier davidson 264.27000427246094 +xavier davidson 288.1999988555908 +xavier ellison 138.42000198364258 +xavier ellison 262.6300048828125 +xavier garcia 148.66000366210938 +xavier hernandez 122.13999938964844 +xavier hernandez 164.97000122070312 +xavier hernandez 306.25 +xavier ichabod 211.84000635147095 +xavier ichabod 244.50000762939453 +xavier johnson 56.53999900817871 +xavier johnson 89.0999984741211 +xavier king 87.22000122070312 +xavier king 151.22999572753906 +xavier laertes 183.65999794006348 +xavier ovid 398.2100067138672 +xavier polk 72.62000274658203 +xavier polk 76.93000030517578 +xavier polk 261.5100030899048 +xavier polk 318.01000213623047 +xavier quirinius 22.1200008392334 +xavier quirinius 89.55000305175781 +xavier quirinius 246.2400016784668 +xavier quirinius 402.2100009918213 +xavier thompson 283.9400005340576 +xavier underhill 120.45000076293945 +xavier white 138.02999591827393 +xavier white 172.06999969482422 +xavier xylophone 79.41999816894531 +xavier zipper 373.67999935150146 +yuri allen 52.849998474121094 +yuri allen 417.3700008392334 +yuri brown 170.52000427246094 +yuri brown 180.70999908447266 +yuri carson 188.99000549316406 +yuri carson 537.6500015258789 +yuri ellison 86.91999816894531 +yuri ellison 376.32999646663666 +yuri falkner 152.99000358581543 +yuri falkner 181.06999969482422 +yuri garcia 274.6800003051758 +yuri hernandez 153.46999764442444 +yuri johnson 197.28000259399414 +yuri johnson 236.0800018310547 +yuri johnson 258.1899985074997 +yuri king 551.9899978637695 +yuri laertes 37.59000015258789 +yuri laertes 253.4799976348877 +yuri nixon 95.54999732971191 +yuri nixon 248.9700005054474 +yuri polk 82.33999633789062 +yuri polk 275.3200035095215 +yuri polk 305.6399974822998 +yuri quirinius 112.97000122070312 +yuri quirinius 148.27999877929688 +yuri quirinius 449.1699924468994 +yuri steinbeck 292.94000244140625 +yuri steinbeck 357.5 +yuri thompson 428.03999519348145 +yuri underhill 83.87000274658203 +yuri underhill 350.7999897003174 +yuri white 132.09000396728516 +yuri xylophone 107.07000160217285 +zach allen 65.43000030517578 +zach brown 135.6999969482422 +zach brown 247.04999542236328 +zach brown 256.8000030517578 +zach brown 362.38000106811523 +zach brown 418.75 +zach carson 291.7700004577637 +zach ellison 135.149995803833 +zach falkner 91.41999816894531 +zach falkner 196.41999912261963 +zach garcia 84.37999725341797 +zach garcia 160.70000457763672 +zach garcia 167.7599983215332 +zach garcia 205.36999893188477 +zach ichabod 116.2699966430664 +zach ichabod 151.18000030517578 +zach king 127.63000106811523 +zach king 182.2699966430664 +zach king 269.0999984741211 +zach miller 199.71000289916992 +zach miller 220.73999977111816 +zach miller 264.0600047111511 +zach ovid 92.55000305175781 +zach ovid 94.33999633789062 +zach ovid 105.94999847561121 +zach ovid 136.04000091552734 +zach quirinius 103.11000061035156 +zach robinson 76.72000122070312 +zach steinbeck 85.48999786376953 +zach steinbeck 182.87000274658203 +zach thompson 116.93999862670898 +zach thompson 319.9499969482422 +zach underhill 86.22000122070312 +zach white 70.52999877929688 +zach xylophone 227.52000427246094 +zach xylophone 286.45000076293945 +zach young 313.00999450683594 +zach zipper 85.87000274658203 +zach zipper 94.43000030517578 +zach zipper 139.38999938964844 diff --git a/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-3-6f104992e0050576085064815de43194 b/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-3-6f104992e0050576085064815de43194 new file mode 100644 index 0000000000000..ae2a1e9dd7d3f --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-3-6f104992e0050576085064815de43194 @@ -0,0 +1,1049 @@ + 43.52666695912679 + 48.735000133514404 + 57.96666590372721 +alice allen 39.21833221117655 +alice allen 42.813999557495116 +alice allen 49.1824996471405 +alice brown 59.68166707456112 +alice carson 46.703333189090095 +alice davidson 59.51833279927572 +alice falkner 64.48333247502644 +alice garcia 55.114999771118164 +alice hernandez 49.61333228151003 +alice hernandez 69.70166667302449 +alice johnson 49.25166805585226 +alice king 20.052499681711197 +alice king 49.019999186197914 +alice king 56.0733331044515 +alice laertes 35.24999898672104 +alice laertes 68.85333251953125 +alice miller 55.542000198364256 +alice nixon 44.010000785191856 +alice nixon 50.900000762939456 +alice nixon 64.78333409627278 +alice ovid 24.90999937057495 +alice polk 47.426666259765625 +alice quirinius 52.80166610081991 +alice quirinius 52.94833393891653 +alice robinson 44.41500027974447 +alice robinson 55.04249954223633 +alice steinbeck 37.735000928243004 +alice steinbeck 41.02800045013428 +alice steinbeck 53.83499916394552 +alice underhill 52.64500021934509 +alice van buren 48.27666727701823 +alice xylophone 28.047500252723694 +alice xylophone 37.82199954986572 +alice xylophone 59.75166606903076 +alice zipper 48.875000635782875 +alice zipper 51.365000108877815 +alice zipper 89.93000030517578 +bob brown 38.134999910990395 +bob brown 56.20666662851969 +bob brown 77.51166598002116 +bob carson 52.77200050354004 +bob davidson 28.945000171661377 +bob davidson 37.946666399637856 +bob davidson 53.90333271026611 +bob ellison 51.383999824523926 +bob ellison 57.30333344141642 +bob ellison 58.53666559855143 +bob ellison 66.7400016784668 +bob falkner 39.053333600362144 +bob garcia 16.705000400543213 +bob garcia 37.16499960422516 +bob garcia 51.17999919255575 +bob garcia 52.37666575113932 +bob garcia 67.39166768391927 +bob hernandez 53.261999893188474 +bob ichabod 43.96999979019165 +bob king 38.03666718800863 +bob king 38.19249892234802 +bob king 67.40999794006348 +bob laertes 21.449999809265137 +bob laertes 50.37166612346967 +bob miller 41.906000471115114 +bob ovid 27.836666425069172 +bob ovid 39.3833335240682 +bob ovid 60.90749979019165 +bob ovid 63.069999313354494 +bob polk 41.88333296775818 +bob quirinius 54.504998207092285 +bob steinbeck 34.33999983469645 +bob van buren 58.29999923706055 +bob white 30.22333288192749 +bob white 41.44000005722046 +bob xylophone 12.163333415985107 +bob xylophone 47.88249921798706 +bob young 27.38499927520752 +bob zipper 27.93800084590912 +bob zipper 59.11800007820129 +bob zipper 66.43000030517578 +calvin allen 71.51000118255615 +calvin brown 50.44166612625122 +calvin brown 53.625000953674316 +calvin brown 67.48333485921223 +calvin carson 62.17750120162964 +calvin davidson 14.03000009059906 +calvin davidson 43.640000343322754 +calvin ellison 52.52750015258789 +calvin falkner 24.016666332880657 +calvin falkner 46.87333329518636 +calvin falkner 47.53666607538859 +calvin falkner 56.47499918937683 +calvin falkner 57.336666107177734 +calvin falkner 72.25 +calvin garcia 46.484999338785805 +calvin hernandez 43.90199909210205 +calvin johnson 76.41500091552734 +calvin laertes 39.09249973297119 +calvin laertes 47.323333422342934 +calvin nixon 30.113332668940227 +calvin nixon 47.7533327738444 +calvin nixon 49.08500075340271 +calvin ovid 41.924999713897705 +calvin ovid 49.10499978065491 +calvin ovid 62.26499938964844 +calvin ovid 62.27999989191691 +calvin polk 52.95333353678385 +calvin quirinius 53.34200019836426 +calvin quirinius 54.831998634338376 +calvin robinson 60.470001220703125 +calvin steinbeck 35.644999980926514 +calvin steinbeck 53.75800037384033 +calvin steinbeck 56.11000084877014 +calvin thompson 42.355000257492065 +calvin thompson 65.91666666666667 +calvin underhill 47.41199951171875 +calvin van buren 31.591666102409363 +calvin van buren 43.18200063705444 +calvin white 45.27500128746033 +calvin white 56.079999923706055 +calvin xylophone 20.394999980926514 +calvin xylophone 40.59999958674113 +calvin xylophone 54.56500005722046 +calvin young 42.246666272481285 +calvin young 64.49000072479248 +calvin zipper 57.49500036239624 +calvin zipper 57.59000039100647 +david allen 40.14999977747599 +david allen 54.545000076293945 +david brown 35.4516666730245 +david brown 62.83999938964844 +david davidson 35.50000067551931 +david davidson 52.75250005722046 +david davidson 74.1500015258789 +david davidson 95.80999755859375 +david ellison 43.43166727821032 +david ellison 52.18750047683716 +david ellison 72.58400039672851 +david hernandez 64.47600173950195 +david ichabod 29.518332719802856 +david ichabod 34.6100010573864 +david laertes 61.127999496459964 +david nixon 53.60249900817871 +david ovid 37.977500796318054 +david ovid 41.58999983469645 +david quirinius 24.96250009536743 +david quirinius 45.23249912261963 +david quirinius 52.282000350952146 +david robinson 42.17750024795532 +david robinson 62.54666519165039 +david thompson 41.88999938964844 +david underhill 48.143333752950035 +david underhill 62.53499941031138 +david underhill 97.55999755859375 +david van buren 34.84749984741211 +david van buren 51.61666679382324 +david white 62.30499839782715 +david xylophone 33.92500019073486 +david xylophone 53.142000222206114 +david xylophone 72.77166684468587 +david young 30.13666756947835 +david young 51.540000915527344 +ethan allen 53.442500591278076 +ethan brown 7.110000133514404 +ethan brown 41.260000228881836 +ethan brown 41.495000084241234 +ethan brown 41.83833312988281 +ethan brown 46.497499227523804 +ethan brown 63.17750072479248 +ethan carson 64.31999969482422 +ethan ellison 46.72599983215332 +ethan ellison 61.24750167876482 +ethan falkner 36.90250104665756 +ethan falkner 52.71000099182129 +ethan garcia 32.771666407585144 +ethan hernandez 56.239999008178714 +ethan johnson 82.38333384195964 +ethan king 8.399999856948853 +ethan laertes 37.93200054168701 +ethan laertes 40.4883329073588 +ethan laertes 53.396667132774986 +ethan laertes 53.81666628519694 +ethan laertes 55.987499713897705 +ethan laertes 66.36999956766765 +ethan laertes 68.83400039672851 +ethan miller 56.519999504089355 +ethan nixon 50.13333353648583 +ethan ovid 37.51333363850912 +ethan polk 2.3499999046325684 +ethan polk 44.147999954223636 +ethan polk 54.91499996185303 +ethan polk 60.340000788370766 +ethan quirinius 40.21750020980835 +ethan quirinius 47.125000298023224 +ethan quirinius 57.69000013669332 +ethan robinson 41.5060001373291 +ethan robinson 55.0640007019043 +ethan underhill 68.01250076293945 +ethan van buren 43.55250072479248 +ethan white 46.40999913215637 +ethan white 53.03200073242188 +ethan xylophone 70.29333432515462 +ethan zipper 46.92999982833862 +ethan zipper 66.04500198364258 +fred davidson 26.487499618902802 +fred davidson 44.99333477020264 +fred davidson 53.93666648864746 +fred ellison 22.2599999109904 +fred ellison 48.047999954223634 +fred ellison 63.423333485921226 +fred falkner 20.035000324249268 +fred falkner 44.243999004364014 +fred falkner 62.130001068115234 +fred hernandez 37.42199997901916 +fred ichabod 46.56999969482422 +fred ichabod 58.275001525878906 +fred johnson 54.63333257039388 +fred king 46.30499919255575 +fred king 61.48500037193298 +fred laertes 26.203333059946697 +fred miller 43.46400032043457 +fred nixon 28.690000534057617 +fred nixon 32.77999955415726 +fred nixon 52.03799936771393 +fred nixon 60.468332290649414 +fred polk 26.81166632970174 +fred polk 42.48999996185303 +fred polk 46.95200061798096 +fred polk 60.465998840332034 +fred quirinius 43.466000366210935 +fred quirinius 45.79799957275391 +fred robinson 62.42833296457926 +fred steinbeck 32.22999954223633 +fred steinbeck 48.73800039291382 +fred steinbeck 65.91750144958496 +fred underhill 72.94250011444092 +fred van buren 41.28250002861023 +fred van buren 41.5283338278532 +fred van buren 50.871665954589844 +fred van buren 81.77000045776367 +fred white 34.897499561309814 +fred young 58.56666692097982 +fred young 70.61499977111816 +fred zipper 33.90500068664551 +gabriella allen 51.041666666666664 +gabriella allen 55.13999938964844 +gabriella brown 54.63333320617676 +gabriella brown 72.33333396911621 +gabriella carson 49.26333236694336 +gabriella davidson 52.72800083160401 +gabriella ellison 54.68999989827474 +gabriella ellison 71.54000091552734 +gabriella falkner 20.834000015258788 +gabriella falkner 46.348333517710365 +gabriella falkner 48.58500099182129 +gabriella garcia 24.78999964396159 +gabriella hernandez 57.61000029246012 +gabriella hernandez 65.11166667938232 +gabriella ichabod 33.78749895095825 +gabriella ichabod 38.9950008392334 +gabriella ichabod 48.15499925613403 +gabriella ichabod 49.739999008178714 +gabriella ichabod 52.789999643961586 +gabriella king 35.58200044631958 +gabriella king 49.63000059723854 +gabriella laertes 47.81000010172526 +gabriella miller 62.17666753133138 +gabriella ovid 45.94333267211914 +gabriella ovid 50.435001373291016 +gabriella polk 42.58000100851059 +gabriella polk 72.87000179290771 +gabriella steinbeck 65.42000102996826 +gabriella steinbeck 72.0499997138977 +gabriella thompson 49.897499561309814 +gabriella thompson 52.84250023961067 +gabriella thompson 57.23249959945679 +gabriella van buren 39.31000053882599 +gabriella van buren 57.920000076293945 +gabriella white 49.85333410898844 +gabriella young 15.369999885559082 +gabriella young 49.04999907811483 +gabriella zipper 42.82499901453654 +gabriella zipper 59.58499972025553 +holly allen 29.50499963760376 +holly brown 55.284998178482056 +holly brown 55.79833386838436 +holly falkner 40.07666663328806 +holly hernandez 46.40500044822693 +holly hernandez 56.14500045776367 +holly hernandez 56.160000483194985 +holly hernandez 58.95166748017073 +holly ichabod 67.19250011444092 +holly ichabod 68.44500064849854 +holly ichabod 90.09000015258789 +holly johnson 42.795000076293945 +holly johnson 55.76999855041504 +holly johnson 66.11333338419597 +holly king 48.788000869750974 +holly king 64.25 +holly laertes 50.40750074386597 +holly miller 45.60666608810425 +holly nixon 58.096665700276695 +holly nixon 69.59249877929688 +holly polk 41.59666601816813 +holly polk 44.64799900054932 +holly robinson 48.405999755859376 +holly thompson 34.605000495910645 +holly thompson 38.36749941110611 +holly thompson 65.99833394338687 +holly underhill 46.52600040435791 +holly underhill 47.92666663726171 +holly underhill 62.41600036621094 +holly underhill 79.95999908447266 +holly van buren 58.63333225250244 +holly white 31.232499361038208 +holly white 61.494998931884766 +holly xylophone 70.32500044504802 +holly young 54.05000114440918 +holly young 57.103333473205566 +holly zipper 50.59500014781952 +holly zipper 67.81000137329102 +irene allen 53.364000701904295 +irene brown 4.789999961853027 +irene brown 51.65399932861328 +irene brown 87.66999816894531 +irene carson 59.2433336575826 +irene ellison 36.446667989095054 +irene ellison 53.85000038146973 +irene falkner 47.429999995231626 +irene falkner 61.20666694641113 +irene garcia 15.369999885559082 +irene garcia 48.28000005086263 +irene garcia 55.643333435058594 +irene ichabod 40.868333299954735 +irene ichabod 64.45666694641113 +irene johnson 44.37999935150147 +irene laertes 27.625 +irene laertes 46.25 +irene laertes 49.17500019073486 +irene miller 75.87500063578288 +irene nixon 29.780000686645508 +irene nixon 30.070000807444256 +irene nixon 36.34333356221517 +irene ovid 27.21500023206075 +irene ovid 55.6175012588501 +irene ovid 60.353999328613284 +irene polk 47.438334147135414 +irene polk 47.77666505177816 +irene polk 51.74200067520142 +irene polk 52.12200088500977 +irene polk 52.184000205993655 +irene quirinius 52.958333015441895 +irene quirinius 55.29000053405762 +irene quirinius 78.79000091552734 +irene robinson 61.398332595825195 +irene steinbeck 64.34000142415364 +irene thompson 41.92250097543001 +irene underhill 27.72499966621399 +irene underhill 50.783331871032715 +irene van buren 46.65999984741211 +irene van buren 49.71000099182129 +irene xylophone 61.220001220703125 +jessica brown 63.449999491373696 +jessica carson 41.78999951481819 +jessica carson 47.52000069618225 +jessica carson 51.83000183105469 +jessica davidson 34.938333332538605 +jessica davidson 45.91000066200892 +jessica davidson 51.082499504089355 +jessica davidson 64.3099988301595 +jessica ellison 32.53000124295553 +jessica ellison 35.265999984741214 +jessica falkner 54.933334032694496 +jessica garcia 32.575000286102295 +jessica garcia 60.584999084472656 +jessica ichabod 46.704999923706055 +jessica johnson 55.73199977874756 +jessica johnson 72.14000066121419 +jessica miller 55.27799835205078 +jessica nixon 58.53999900817871 +jessica nixon 90.06999969482422 +jessica ovid 36.47500014305115 +jessica ovid 63.03000005086263 +jessica polk 69.52999877929688 +jessica quirinius 19.01333288351695 +jessica quirinius 41.630000829696655 +jessica quirinius 42.58750009536743 +jessica quirinius 47.72999986012777 +jessica robinson 64.81000061035157 +jessica thompson 45.83666737874349 +jessica thompson 57.540000915527344 +jessica underhill 45.360000133514404 +jessica underhill 51.40999889373779 +jessica underhill 64.27250099182129 +jessica van buren 9.739999771118164 +jessica white 38.38999891281128 +jessica white 40.75400023460388 +jessica white 61.89800128936768 +jessica white 62.34749984741211 +jessica white 65.7680004119873 +jessica xylophone 50.808332761128746 +jessica young 18.400000254313152 +jessica young 49.04400033950806 +jessica zipper 35.039999643961586 +jessica zipper 52.78999869028727 +jessica zipper 55.51499891281128 +katie allen 56.10800056457519 +katie brown 48.86833222707113 +katie davidson 96.91000366210938 +katie ellison 31.106667200724285 +katie ellison 38.356666127840676 +katie falkner 17.24333318074544 +katie garcia 53.429999033610024 +katie garcia 53.933334002892174 +katie hernandez 72.71666717529297 +katie ichabod 39.97999954223633 +katie ichabod 54.99599914550781 +katie ichabod 59.41999944051107 +katie king 44.88599967956543 +katie king 60.2549991607666 +katie king 67.27499961853027 +katie miller 31.399999618530273 +katie miller 50.57666619618734 +katie nixon 23.190000534057617 +katie ovid 52.96000158786774 +katie polk 51.029998779296875 +katie polk 54.93600006103516 +katie robinson 13.890000343322754 +katie van buren 52.78999908765157 +katie van buren 53.21750092506409 +katie white 59.799998950958255 +katie white 64.70499992370605 +katie xylophone 53.54499959945679 +katie young 31.010000228881836 +katie young 37.55500102043152 +katie young 49.87499872843424 +katie zipper 29.79666694998741 +katie zipper 50.95000076293945 +luke allen 43.757998657226565 +luke allen 53.18000030517578 +luke allen 62.09749984741211 +luke allen 70.2933349609375 +luke allen 89.55000305175781 +luke brown 45.95999972025553 +luke davidson 28.950000762939453 +luke davidson 53.20500040054321 +luke ellison 5.159999907016754 +luke ellison 18.630000392595928 +luke ellison 76.88000106811523 +luke falkner 6.175000190734863 +luke falkner 54.0049991607666 +luke garcia 15.320000424981117 +luke garcia 25.47000026702881 +luke ichabod 33.950000286102295 +luke ichabod 57.17500114440918 +luke johnson 14.4399995803833 +luke johnson 37.23333263397217 +luke johnson 52.660000801086426 +luke laertes 11.819999694824219 +luke laertes 23.5 +luke laertes 43.71999907493591 +luke laertes 45.9900016784668 +luke laertes 57.85499906539917 +luke miller 67.85000038146973 +luke ovid 38.04999923706055 +luke ovid 79.64200134277344 +luke polk 55.106666564941406 +luke polk 70.18999862670898 +luke quirinius 26.016666491826374 +luke robinson 61.256665547688804 +luke robinson 72.61999893188477 +luke thompson 47.203332940737404 +luke underhill 49.244999408721924 +luke underhill 59.32999897003174 +luke underhill 72.18333307902019 +luke van buren 64.646666208903 +luke white 67.12000274658203 +luke xylophone 46.74333190917969 +luke zipper 36.16000008583069 +mike allen 10.709999859333038 +mike brown 61.0674991607666 +mike carson 50.90249824523926 +mike carson 52.30200061798096 +mike carson 52.51499938964844 +mike davidson 29.733333269755047 +mike davidson 47.396666526794434 +mike ellison 37.807498931884766 +mike ellison 44.339999198913574 +mike ellison 45.89199924468994 +mike ellison 47.91249990463257 +mike ellison 55.295000076293945 +mike falkner 16.479999542236328 +mike garcia 38.98999913533529 +mike garcia 54.91333325703939 +mike garcia 57.096666971842446 +mike hernandez 59.45000076293945 +mike hernandez 62.446667989095054 +mike ichabod 54.69666544596354 +mike king 38.790000915527344 +mike king 43.2050017118454 +mike king 46.85666592915853 +mike king 46.945000648498535 +mike king 53.382500648498535 +mike king 94.68000030517578 +mike miller 3.9600000381469727 +mike nixon 53.894999742507935 +mike nixon 92.95999908447266 +mike polk 12.449999809265137 +mike polk 39.040000319480896 +mike polk 71.40500068664551 +mike quirinius 89.37999725341797 +mike steinbeck 34.05499875545502 +mike steinbeck 45.909999465942384 +mike steinbeck 51.286667346954346 +mike steinbeck 59.292500019073486 +mike van buren 43.13999819755554 +mike van buren 49.59000015258789 +mike white 30.53999964396159 +mike white 34.61499961217245 +mike white 48.08999943733215 +mike white 53.967501163482666 +mike young 10.484999895095825 +mike young 47.070000648498535 +mike young 74.58999633789062 +mike zipper 29.029999288419884 +mike zipper 77.96200027465821 +mike zipper 91.57500076293945 +nick allen 29.146666367848713 +nick allen 64.4325008392334 +nick brown 49.429999669392906 +nick davidson 34.36599922180176 +nick ellison 49.345001220703125 +nick ellison 89.01000213623047 +nick falkner 7.820000171661377 +nick falkner 45.025000631809235 +nick garcia 23.9499994913737 +nick garcia 33.49333349863688 +nick garcia 64.33499908447266 +nick ichabod 30.945000171661377 +nick ichabod 56.27499961853027 +nick ichabod 59.24249863624573 +nick johnson 32.3674995303154 +nick johnson 74.30666859944661 +nick laertes 38.38749980926514 +nick miller 82.97000122070312 +nick nixon 70.01333173116048 +nick ovid 56.82999897003174 +nick polk 33.929999669392906 +nick quirinius 58.91199932098389 +nick quirinius 81.16999816894531 +nick robinson 34.53999948501587 +nick robinson 59.80499839782715 +nick steinbeck 57.25333340962728 +nick thompson 51.3725004196167 +nick underhill 55.47333335876465 +nick van buren 74.21666717529297 +nick xylophone 51.82000001271566 +nick young 0.27000001072883606 +nick young 41.02499961853027 +nick zipper 47.72999954223633 +nick zipper 61.917999267578125 +oscar allen 35.21600015163422 +oscar brown 38.03999948501587 +oscar carson 38.09600009918213 +oscar carson 51.34333419799805 +oscar carson 56.9925012588501 +oscar carson 74.00500106811523 +oscar carson 98.51000213623047 +oscar davidson 65.88750076293945 +oscar ellison 50.507500410079956 +oscar ellison 66.57000096638997 +oscar falkner 64.42000198364258 +oscar garcia 66.36333465576172 +oscar hernandez 42.089999516805015 +oscar hernandez 51.21199997067451 +oscar ichabod 25.300000190734863 +oscar ichabod 41.173332850138344 +oscar ichabod 53.44666830698649 +oscar ichabod 71.80000305175781 +oscar johnson 39.9366668065389 +oscar johnson 44.30500118434429 +oscar king 30.59749937057495 +oscar king 46.149999141693115 +oscar king 49.65999984741211 +oscar laertes 5.510000228881836 +oscar laertes 19.099999745686848 +oscar laertes 39.36250039935112 +oscar laertes 45.340000788370766 +oscar nixon 41.619998931884766 +oscar ovid 45.09000015258789 +oscar ovid 82.23999786376953 +oscar ovid 83.47750091552734 +oscar polk 42.54333368937174 +oscar polk 56.8033332824707 +oscar quirinius 52.94500160217285 +oscar quirinius 65.34666570027669 +oscar quirinius 67.00000127156575 +oscar quirinius 79.4800033569336 +oscar robinson 32.16999944051107 +oscar robinson 38.880001068115234 +oscar robinson 63.9433339436849 +oscar robinson 65.80500030517578 +oscar steinbeck 45.73599967956543 +oscar thompson 40.33599853515625 +oscar thompson 47.860000928243004 +oscar thompson 60.529998779296875 +oscar thompson 60.59333292643229 +oscar underhill 43.980000764131546 +oscar van buren 2.180000066757202 +oscar van buren 53.34999942779541 +oscar van buren 61.880001068115234 +oscar white 38.0633331934611 +oscar white 42.584999084472656 +oscar white 55.179999033610024 +oscar white 74.4900016784668 +oscar xylophone 41.3133331934611 +oscar xylophone 55.5049991607666 +oscar xylophone 67.54500007629395 +oscar zipper 15.680000305175781 +oscar zipper 24.019999504089355 +oscar zipper 39.81999969482422 +priscilla brown 42.88666502634684 +priscilla brown 63.242499351501465 +priscilla brown 77.56999969482422 +priscilla carson 44.799999713897705 +priscilla carson 49.663333892822266 +priscilla carson 78.57333374023438 +priscilla ichabod 56.83666737874349 +priscilla ichabod 58.48666636149088 +priscilla johnson 34.416667779286705 +priscilla johnson 52.890000343322754 +priscilla johnson 53.72666883468628 +priscilla johnson 61.939998626708984 +priscilla johnson 89.1500015258789 +priscilla king 34.30750045180321 +priscilla nixon 27.734999656677246 +priscilla nixon 60.90999984741211 +priscilla ovid 48.13500142097473 +priscilla ovid 66.35999870300293 +priscilla polk 15.149999618530273 +priscilla quirinius 18.606666564941406 +priscilla thompson 48.87000131607056 +priscilla underhill 40.05500078201294 +priscilla underhill 49.54999961853027 +priscilla van buren 42.62666734059652 +priscilla van buren 61.54000017642975 +priscilla van buren 72.80500030517578 +priscilla white 43.177499771118164 +priscilla xylophone 21.489999771118164 +priscilla xylophone 40.144999124109745 +priscilla xylophone 59.61000061035156 +priscilla young 31.610000324249267 +priscilla young 53.71000152826309 +priscilla zipper 18.8799991607666 +priscilla zipper 25.670000076293945 +quinn allen 56.77749991416931 +quinn allen 76.47250080108643 +quinn brown 23.536666870117188 +quinn brown 31.829999446868896 +quinn brown 50.388000297546384 +quinn davidson 41.42499923706055 +quinn davidson 45.90250015258789 +quinn davidson 48.16000032424927 +quinn davidson 71.51000022888184 +quinn ellison 50.6766668955485 +quinn ellison 62.32666842142741 +quinn garcia 39.98599967956543 +quinn garcia 43.27000045776367 +quinn garcia 74.31999969482422 +quinn garcia 92.33000183105469 +quinn ichabod 44.81999969482422 +quinn king 50.99666786193848 +quinn king 74.62000274658203 +quinn laertes 5.884999990463257 +quinn laertes 49.1379997253418 +quinn laertes 56.18000030517578 +quinn nixon 74.66500091552734 +quinn ovid 1.2100000381469727 +quinn quirinius 46.15500068664551 +quinn robinson 44.96249985694885 +quinn steinbeck 24.06999921798706 +quinn steinbeck 41.750000953674316 +quinn thompson 23.744999408721924 +quinn thompson 73.64500045776367 +quinn underhill 41.83333269755045 +quinn underhill 49.63749885559082 +quinn underhill 83.85000228881836 +quinn van buren 54.36333338419596 +quinn young 45.489999771118164 +quinn zipper 22.25 +quinn zipper 33.355000019073486 +rachel allen 15.8100004196167 +rachel allen 71.57666524251302 +rachel brown 2.9600000381469727 +rachel brown 33.022499561309814 +rachel brown 42.442498207092285 +rachel brown 59.21999931335449 +rachel brown 64.52666727701823 +rachel carson 61.17999776204427 +rachel carson 69.85333315531413 +rachel davidson 42.292500495910645 +rachel ellison 10.600000381469727 +rachel falkner 23.615000784397125 +rachel falkner 50.46250069141388 +rachel falkner 54.669999837875366 +rachel falkner 72.96999740600586 +rachel johnson 61.76249885559082 +rachel king 22.005000591278076 +rachel king 66.46500015258789 +rachel laertes 39.025001525878906 +rachel laertes 45.45000076293945 +rachel ovid 0.6000000238418579 +rachel ovid 1.0800000429153442 +rachel polk 78.08499908447266 +rachel quirinius 51.12000131607056 +rachel robinson 30.360000610351562 +rachel robinson 37.683333237965904 +rachel robinson 39.199998219807945 +rachel thompson 0.5600000023841858 +rachel thompson 38.81000010172526 +rachel thompson 49.772499561309814 +rachel underhill 48.45000076293945 +rachel white 42.83999991416931 +rachel white 58.609999656677246 +rachel young 43.130001068115234 +rachel zipper 7.059999942779541 +rachel zipper 49.676667173703514 +sarah carson 54.022500067949295 +sarah carson 54.18333212534586 +sarah carson 87.81000137329102 +sarah ellison 16.989999771118164 +sarah falkner 73.06500005722046 +sarah falkner 99.36000061035156 +sarah garcia 38.43250036239624 +sarah garcia 64.97333272298177 +sarah garcia 73.6500015258789 +sarah ichabod 57.46000003814697 +sarah ichabod 81.31999969482422 +sarah johnson 34.98499917984009 +sarah johnson 45.099998474121094 +sarah johnson 74.42249870300293 +sarah johnson 74.72000122070312 +sarah king 41.869998931884766 +sarah king 48.25 +sarah miller 41.53499984741211 +sarah ovid 33.38000011444092 +sarah robinson 33.83000183105469 +sarah robinson 66.88999938964844 +sarah steinbeck 31.023332993189495 +sarah white 45.974998474121094 +sarah white 61.54666709899902 +sarah xylophone 61.60666529337565 +sarah young 45.560001373291016 +sarah zipper 60.69000053405762 +tom brown 40.08000135421753 +tom brown 55.44499969482422 +tom carson 5.440000057220459 +tom carson 26.32499885559082 +tom carson 31.476666768391926 +tom davidson 53.00749921798706 +tom ellison 67.32666714986165 +tom ellison 76.73999786376953 +tom ellison 77.29499816894531 +tom falkner 60.130001068115234 +tom falkner 88.22000122070312 +tom hernandez 41.36000061035156 +tom hernandez 81.63999938964844 +tom ichabod 42.08666547139486 +tom johnson 33.75999959309896 +tom johnson 53.47666549682617 +tom king 40.0 +tom laertes 32.05000114440918 +tom laertes 43.46000099182129 +tom miller 21.229999542236328 +tom miller 43.37666702270508 +tom miller 44.41333325703939 +tom nixon 46.35333251953125 +tom ovid 43.04499912261963 +tom polk 38.29999923706055 +tom polk 54.45000076293945 +tom quirinius 19.82000058889389 +tom quirinius 22.580000400543213 +tom robinson 66.16999816894531 +tom robinson 74.18666712443034 +tom robinson 80.77000045776367 +tom robinson 98.72000122070312 +tom steinbeck 44.999999046325684 +tom van buren 31.103334546089172 +tom van buren 40.779998779296875 +tom van buren 63.5099983215332 +tom white 49.06500053405762 +tom young 54.16999912261963 +tom young 78.54999923706055 +tom zipper 48.666666984558105 +ulysses brown 72.79000091552734 +ulysses carson 40.28500175476074 +ulysses carson 71.55000305175781 +ulysses carson 77.41999816894531 +ulysses carson 79.54000091552734 +ulysses davidson 41.90166711807251 +ulysses ellison 96.7300033569336 +ulysses garcia 89.80000305175781 +ulysses hernandez 21.339999516805012 +ulysses hernandez 54.470001220703125 +ulysses hernandez 80.11000061035156 +ulysses ichabod 19.1299991607666 +ulysses ichabod 98.56999969482422 +ulysses johnson 42.55000019073486 +ulysses king 81.57000223795573 +ulysses laertes 1.9199999570846558 +ulysses laertes 11.890000343322754 +ulysses laertes 53.599998474121094 +ulysses miller 40.43499946594238 +ulysses miller 47.78200054168701 +ulysses nixon 74.60333251953125 +ulysses ovid 30.940000534057617 +ulysses polk 8.710000038146973 +ulysses polk 39.17500019073486 +ulysses polk 65.0199966430664 +ulysses polk 68.41333389282227 +ulysses quirinius 65.93499946594238 +ulysses robinson 79.48999786376953 +ulysses steinbeck 32.40999984741211 +ulysses steinbeck 43.93499994277954 +ulysses thompson 79.96000289916992 +ulysses underhill 17.85000006357829 +ulysses underhill 42.5533332824707 +ulysses underhill 46.60000157356262 +ulysses underhill 47.84000049829483 +ulysses underhill 51.85000133514404 +ulysses underhill 67.77999877929688 +ulysses underhill 99.66999816894531 +ulysses van buren 69.89999961853027 +ulysses white 45.01500183343887 +ulysses white 71.3933334350586 +ulysses xylophone 27.880000591278076 +ulysses xylophone 39.689998626708984 +ulysses xylophone 54.099998474121094 +ulysses young 32.52000045776367 +ulysses young 39.81333382924398 +ulysses young 80.7933349609375 +victor allen 44.27000045776367 +victor allen 57.994998931884766 +victor brown 56.84499931335449 +victor brown 59.34000015258789 +victor brown 90.37999725341797 +victor brown 91.97000122070312 +victor davidson 52.07000160217285 +victor davidson 54.239999771118164 +victor davidson 68.8033332824707 +victor ellison 45.45750088989735 +victor ellison 58.42999839782715 +victor hernandez 23.164999961853027 +victor hernandez 57.98499870300293 +victor hernandez 59.619998931884766 +victor hernandez 71.42499923706055 +victor hernandez 80.19499969482422 +victor johnson 1.5800000429153442 +victor johnson 46.7450008392334 +victor johnson 72.59000015258789 +victor king 37.559998989105225 +victor king 47.880001068115234 +victor laertes 50.0099983215332 +victor laertes 70.13999938964844 +victor miller 22.1200008392334 +victor nixon 34.029998779296875 +victor nixon 37.08500003814697 +victor ovid 18.815000653266907 +victor polk 3.0 +victor quirinius 26.450000405311584 +victor quirinius 33.080001533031464 +victor robinson 4.590000152587891 +victor robinson 41.21999931335449 +victor steinbeck 41.82500123977661 +victor steinbeck 46.09000015258789 +victor steinbeck 52.720001220703125 +victor thompson 45.346666971842446 +victor van buren 34.970001220703125 +victor van buren 46.57333246866862 +victor white 5.670000076293945 +victor white 74.16999816894531 +victor xylophone 11.220000267028809 +victor xylophone 28.954999923706055 +victor xylophone 34.010000228881836 +victor xylophone 43.179999351501465 +victor xylophone 62.38999938964844 +victor young 70.91000175476074 +victor zipper 48.795000076293945 +wendy allen 0.6100000143051147 +wendy allen 52.64999961853027 +wendy allen 56.06999969482422 +wendy brown 51.874999046325684 +wendy brown 66.73999913533528 +wendy ellison 27.014999389648438 +wendy ellison 94.66000366210938 +wendy falkner 14.425000190734863 +wendy falkner 77.36000061035156 +wendy falkner 85.68000030517578 +wendy garcia 4.409999847412109 +wendy garcia 20.390000343322754 +wendy garcia 38.42500060796738 +wendy garcia 57.25 +wendy hernandez 48.11000061035156 +wendy ichabod 13.149999618530273 +wendy king 33.234999656677246 +wendy king 74.97999954223633 +wendy king 87.94000244140625 +wendy laertes 49.01999855041504 +wendy laertes 54.750000635782875 +wendy laertes 79.98999786376953 +wendy miller 1.2699999809265137 +wendy miller 47.41500186920166 +wendy nixon 30.17999916151166 +wendy nixon 45.91999816894531 +wendy ovid 85.26000213623047 +wendy ovid 86.62999725341797 +wendy polk 43.679999669392906 +wendy polk 54.65333424011866 +wendy quirinius 12.15999984741211 +wendy quirinius 14.300000190734863 +wendy robinson 58.815001487731934 +wendy robinson 71.06999969482422 +wendy robinson 71.77999877929688 +wendy steinbeck 77.31500053405762 +wendy thompson 58.09000015258789 +wendy thompson 85.76000213623047 +wendy underhill 41.23333231608073 +wendy underhill 49.33500003814697 +wendy underhill 81.56000137329102 +wendy van buren 57.459999084472656 +wendy van buren 91.43500137329102 +wendy white 73.68000030517578 +wendy xylophone 31.149999618530273 +wendy xylophone 76.81499862670898 +wendy young 8.449999809265137 +wendy young 33.7599983215332 +xavier allen 45.68000030517578 +xavier allen 52.255000591278076 +xavier allen 83.93000030517578 +xavier brown 7.789999961853027 +xavier brown 77.15500259399414 +xavier brown 80.35000228881836 +xavier carson 20.790000915527344 +xavier carson 55.30000019073486 +xavier davidson 15.920000076293945 +xavier davidson 58.46999931335449 +xavier davidson 82.41000366210938 +xavier ellison 16.614999771118164 +xavier ellison 77.97000122070312 +xavier garcia 49.415000915527344 +xavier hernandez 6.670000076293945 +xavier hernandez 23.054999828338623 +xavier hernandez 67.26000213623047 +xavier ichabod 4.71999979019165 +xavier ichabod 71.19000244140625 +xavier johnson 27.299999237060547 +xavier johnson 89.0999984741211 +xavier king 22.729999542236328 +xavier king 87.22000122070312 +xavier laertes 24.050000190734863 +xavier ovid 58.08000183105469 +xavier polk 13.869999885559082 +xavier polk 58.98750066757202 +xavier polk 72.62000274658203 +xavier polk 76.93000030517578 +xavier quirinius 22.1200008392334 +xavier quirinius 58.24666786193848 +xavier quirinius 62.52000045776367 +xavier quirinius 89.55000305175781 +xavier thompson 9.930000305175781 +xavier underhill 47.27000045776367 +xavier white 8.369999885559082 +xavier white 75.29000091552734 +xavier xylophone 79.41999816894531 +xavier zipper 8.449999809265137 +yuri allen 30.6299991607666 +yuri allen 52.849998474121094 +yuri brown 75.19000244140625 +yuri brown 84.02999877929688 +yuri carson 6.289999961853027 +yuri carson 91.16000366210938 +yuri ellison 1.1200000047683716 +yuri ellison 86.91999816894531 +yuri falkner 6.739999771118164 +yuri falkner 80.8650016784668 +yuri garcia 27.65999984741211 +yuri hernandez 2.069999933242798 +yuri johnson 0.12999999523162842 +yuri johnson 39.900001525878906 +yuri johnson 48.220001220703125 +yuri king 22.270000457763672 +yuri laertes 10.15999984741211 +yuri laertes 37.59000015258789 +yuri nixon 2.200000047683716 +yuri nixon 17.3700008392334 +yuri polk 26.760000228881836 +yuri polk 28.790000915527344 +yuri polk 82.33999633789062 +yuri quirinius 10.260000228881836 +yuri quirinius 38.69000053405762 +yuri quirinius 57.93000030517578 +yuri steinbeck 56.064998626708984 +yuri steinbeck 75.87999725341797 +yuri thompson 14.920000076293945 +yuri underhill 23.770000457763672 +yuri underhill 83.87000274658203 +yuri white 34.58000183105469 +yuri xylophone 20.3799991607666 +zach allen 65.43000030517578 +zach brown 48.0099983215332 +zach brown 57.08000183105469 +zach brown 58.24999809265137 +zach brown 67.37999725341797 +zach brown 75.7300033569336 +zach carson 67.78500175476074 +zach ellison 6.840000152587891 +zach falkner 50.274999141693115 +zach falkner 91.41999816894531 +zach garcia 32.20000076293945 +zach garcia 35.79999923706055 +zach garcia 69.97000122070312 +zach garcia 84.37999725341797 +zach ichabod 36.88999938964844 +zach ichabod 64.25 +zach king 31.864999771118164 +zach king 46.18000030517578 +zach king 86.93000030517578 +zach miller 2.5999999046325684 +zach miller 21.280000686645508 +zach miller 53.27000045776367 +zach ovid 0.10000000149011612 +zach ovid 23.06999969482422 +zach ovid 92.55000305175781 +zach ovid 94.33999633789062 +zach quirinius 39.209999084472656 +zach robinson 76.72000122070312 +zach steinbeck 85.48999786376953 +zach steinbeck 90.05000305175781 +zach thompson 53.59000015258789 +zach thompson 71.5 +zach underhill 86.22000122070312 +zach white 70.52999877929688 +zach xylophone 29.40999984741211 +zach xylophone 71.01000213623047 +zach young 71.31999969482422 +zach zipper 7.539999961853027 +zach zipper 85.87000274658203 +zach zipper 94.43000030517578 diff --git a/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-4-cd2e3d2344810cb3ba843d4c01c81d7e b/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-4-cd2e3d2344810cb3ba843d4c01c81d7e new file mode 100644 index 0000000000000..ee1c26e331a1e --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-4-cd2e3d2344810cb3ba843d4c01c81d7e @@ -0,0 +1,1049 @@ + 17.601666666666667 + 30.72 + 33.07833333333334 +alice allen 23.081666666666667 +alice allen 23.263333333333332 +alice allen 31.38 +alice brown 11.518333333333333 +alice carson 31.99166666666667 +alice davidson 26.986666666666665 +alice falkner 30.513333333333335 +alice garcia 17.13 +alice hernandez 28.126666666666665 +alice hernandez 28.69666666666667 +alice johnson 28.30333333333333 +alice king 23.451666666666668 +alice king 25.20333333333333 +alice king 30.613999999999997 +alice laertes 23.633333333333336 +alice laertes 30.16428571428571 +alice miller 32.50833333333333 +alice nixon 25.278333333333336 +alice nixon 35.345 +alice nixon 36.458333333333336 +alice ovid 26.04714285714286 +alice polk 23.185 +alice quirinius 20.388333333333335 +alice quirinius 23.064999999999998 +alice robinson 30.296666666666667 +alice robinson 33.15 +alice steinbeck 27.894999999999996 +alice steinbeck 31.47833333333334 +alice steinbeck 36.089999999999996 +alice underhill 22.513333333333335 +alice van buren 36.32000000000001 +alice xylophone 27.355000000000004 +alice xylophone 30.505714285714284 +alice xylophone 30.613999999999997 +alice zipper 20.113333333333333 +alice zipper 28.058333333333337 +alice zipper 29.705000000000002 +bob brown 13.546666666666669 +bob brown 16.50333333333333 +bob brown 27.59 +bob carson 25.781666666666666 +bob davidson 18.073333333333334 +bob davidson 23.573333333333334 +bob davidson 31.894999999999996 +bob ellison 16.493333333333336 +bob ellison 17.889999999999997 +bob ellison 25.84142857142857 +bob ellison 33.07833333333334 +bob falkner 21.783333333333335 +bob garcia 16.492857142857144 +bob garcia 23.088333333333335 +bob garcia 26.42833333333333 +bob garcia 32.709999999999994 +bob garcia 33.91833333333334 +bob hernandez 22.303333333333338 +bob ichabod 23.336666666666662 +bob king 20.591666666666665 +bob king 21.244999999999997 +bob king 33.15333333333333 +bob laertes 21.240000000000002 +bob laertes 28.88 +bob miller 32.158750000000005 +bob ovid 21.83666666666667 +bob ovid 23.678571428571427 +bob ovid 25.12333333333333 +bob ovid 31.46 +bob polk 16.3475 +bob quirinius 28.465714285714284 +bob steinbeck 26.99 +bob van buren 26.127142857142854 +bob white 23.021666666666672 +bob white 23.582857142857144 +bob xylophone 26.18166666666667 +bob xylophone 27.995 +bob young 25.495714285714286 +bob zipper 26.435000000000002 +bob zipper 28.384285714285713 +bob zipper 30.65833333333333 +calvin allen 20.812 +calvin brown 20.808 +calvin brown 28.26 +calvin brown 28.37875 +calvin carson 20.38714285714286 +calvin davidson 22.65142857142857 +calvin davidson 23.585714285714285 +calvin ellison 24.6875 +calvin falkner 16.073333333333334 +calvin falkner 17.054285714285715 +calvin falkner 25.2025 +calvin falkner 28.525714285714283 +calvin falkner 33.382 +calvin falkner 37.29666666666667 +calvin garcia 19.924285714285713 +calvin hernandez 21.759999999999998 +calvin johnson 24.03222222222222 +calvin laertes 24.702857142857145 +calvin laertes 28.788333333333338 +calvin nixon 15.631250000000001 +calvin nixon 25.64428571428571 +calvin nixon 29.084999999999997 +calvin ovid 16.243333333333332 +calvin ovid 25.2025 +calvin ovid 25.935555555555553 +calvin ovid 29.299999999999997 +calvin polk 29.615 +calvin quirinius 19.294999999999998 +calvin quirinius 31.64625 +calvin robinson 27.811428571428575 +calvin steinbeck 17.14 +calvin steinbeck 17.535 +calvin steinbeck 21.551666666666666 +calvin thompson 28.592857142857145 +calvin thompson 33.382 +calvin underhill 20.113333333333333 +calvin van buren 28.384285714285713 +calvin van buren 32.106 +calvin white 26.948333333333334 +calvin white 28.256249999999998 +calvin xylophone 25.33666666666667 +calvin xylophone 27.061428571428568 +calvin xylophone 28.384285714285713 +calvin young 19.331666666666667 +calvin young 24.63 +calvin zipper 16.08125 +calvin zipper 28.80555555555556 +david allen 26.21375 +david allen 26.948333333333334 +david brown 14.222 +david brown 27.1 +david davidson 21.062857142857144 +david davidson 26.12 +david davidson 30.951428571428576 +david davidson 31.96142857142857 +david ellison 22.687142857142856 +david ellison 23.435 +david ellison 25.442999999999998 +david hernandez 28.279999999999998 +david ichabod 16.3475 +david ichabod 23.751428571428573 +david laertes 29.07 +david nixon 25.535714285714285 +david ovid 26.75714285714286 +david ovid 29.912857142857142 +david quirinius 17.179999999999996 +david quirinius 20.808 +david quirinius 22.90625 +david robinson 25.274 +david robinson 25.885 +david thompson 22.19125 +david underhill 20.812 +david underhill 21.546666666666667 +david underhill 28.26 +david van buren 24.472857142857148 +david van buren 32.75125 +david white 22.861428571428576 +david xylophone 19.3325 +david xylophone 26.930000000000003 +david xylophone 30.093333333333334 +david young 21.12375 +david young 25.03857142857143 +ethan allen 27.038333333333338 +ethan brown 16.3475 +ethan brown 19.37 +ethan brown 23.73 +ethan brown 25.57166666666667 +ethan brown 26.168333333333333 +ethan brown 31.893333333333334 +ethan carson 25.655714285714286 +ethan ellison 27.807777777777776 +ethan ellison 32.244285714285716 +ethan falkner 28.287142857142857 +ethan falkner 31.411428571428576 +ethan garcia 24.153750000000002 +ethan hernandez 17.986 +ethan johnson 31.54571428571429 +ethan king 22.62 +ethan laertes 15.045555555555556 +ethan laertes 17.889999999999997 +ethan laertes 24.28 +ethan laertes 25.2025 +ethan laertes 26.56888888888889 +ethan laertes 28.37875 +ethan laertes 34.84571428571429 +ethan miller 28.592857142857145 +ethan nixon 29.86833333333333 +ethan ovid 17.85166666666667 +ethan polk 16.463 +ethan polk 18.184545454545457 +ethan polk 25.737142857142857 +ethan polk 29.895000000000003 +ethan quirinius 21.827777777777776 +ethan quirinius 23.42 +ethan quirinius 35.54333333333333 +ethan robinson 24.03222222222222 +ethan robinson 36.35333333333333 +ethan underhill 21.545555555555556 +ethan van buren 15.21142857142857 +ethan white 29.702857142857145 +ethan white 33.33 +ethan xylophone 29.521666666666665 +ethan zipper 23.994 +ethan zipper 26.765 +fred davidson 27.729999999999997 +fred davidson 29.90625 +fred davidson 30.804999999999996 +fred ellison 16.720000000000002 +fred ellison 22.17125 +fred ellison 33.15833333333334 +fred falkner 17.13142857142857 +fred falkner 26.82 +fred falkner 31.925555555555555 +fred hernandez 28.094285714285718 +fred ichabod 23.352500000000003 +fred ichabod 32.906666666666666 +fred johnson 16.9925 +fred king 20.125 +fred king 30.377142857142854 +fred laertes 26.458571428571428 +fred miller 29.66666666666667 +fred nixon 19.565555555555555 +fred nixon 21.830000000000002 +fred nixon 25.828333333333333 +fred nixon 28.094285714285718 +fred polk 21.744999999999997 +fred polk 22.301666666666666 +fred polk 24.156666666666666 +fred polk 28.217142857142857 +fred quirinius 27.495 +fred quirinius 29.615714285714283 +fred robinson 24.243636363636366 +fred steinbeck 17.91333333333333 +fred steinbeck 21.12375 +fred steinbeck 26.47125 +fred underhill 26.43166666666667 +fred van buren 16.165714285714284 +fred van buren 23.285 +fred van buren 24.875714285714288 +fred van buren 27.878333333333334 +fred white 29.498571428571427 +fred young 17.889999999999997 +fred young 19.565555555555555 +fred zipper 21.581428571428575 +gabriella allen 22.03142857142857 +gabriella allen 26.87375 +gabriella brown 29.675714285714285 +gabriella brown 32.44 +gabriella carson 28.75 +gabriella davidson 27.531250000000004 +gabriella ellison 19.82 +gabriella ellison 27.353333333333335 +gabriella falkner 17.535 +gabriella falkner 19.487500000000004 +gabriella falkner 30.964999999999996 +gabriella garcia 20.544 +gabriella hernandez 20.818333333333335 +gabriella hernandez 28.094285714285718 +gabriella ichabod 10.58 +gabriella ichabod 18.64272727272727 +gabriella ichabod 20.504444444444445 +gabriella ichabod 23.185 +gabriella ichabod 23.35375 +gabriella king 16.18 +gabriella king 27.887500000000003 +gabriella laertes 23.799090909090907 +gabriella miller 15.695714285714283 +gabriella ovid 23.515454545454546 +gabriella ovid 33.33 +gabriella polk 20.38714285714286 +gabriella polk 35.77285714285715 +gabriella steinbeck 16.18 +gabriella steinbeck 32.464999999999996 +gabriella thompson 26.477777777777778 +gabriella thompson 27.29777777777778 +gabriella thompson 30.87666666666667 +gabriella van buren 28.513333333333335 +gabriella van buren 32.41111111111111 +gabriella white 26.765 +gabriella young 24.022499999999997 +gabriella young 29.521666666666665 +gabriella zipper 21.43727272727273 +gabriella zipper 32.106 +holly allen 24.271428571428572 +holly brown 22.959 +holly brown 27.498 +holly falkner 29.66666666666667 +holly hernandez 14.179999999999998 +holly hernandez 22.396666666666665 +holly hernandez 27.434000000000005 +holly hernandez 27.887500000000003 +holly ichabod 27.87375 +holly ichabod 32.525 +holly ichabod 34.042857142857144 +holly johnson 20.808 +holly johnson 25.024285714285718 +holly johnson 30.487142857142857 +holly king 23.185 +holly king 29.008888888888894 +holly laertes 19.41 +holly miller 29.89333333333333 +holly nixon 23.162857142857142 +holly nixon 28.876250000000002 +holly polk 22.7825 +holly polk 26.297499999999996 +holly robinson 24.160000000000004 +holly thompson 19.565555555555555 +holly thompson 27.048999999999996 +holly thompson 29.70555555555556 +holly underhill 17.876250000000002 +holly underhill 27.820000000000004 +holly underhill 30.613999999999997 +holly underhill 30.708 +holly van buren 20.113333333333333 +holly white 23.185 +holly white 29.64272727272727 +holly xylophone 26.400909090909092 +holly young 27.807777777777776 +holly young 31.63 +holly zipper 27.401999999999997 +holly zipper 28.384285714285713 +irene allen 35.345 +irene brown 22.527500000000003 +irene brown 28.384285714285713 +irene brown 32.81875 +irene carson 27.10666666666667 +irene ellison 16.720000000000002 +irene ellison 28.592857142857145 +irene falkner 19.41 +irene falkner 30.564999999999998 +irene garcia 16.9925 +irene garcia 24.03222222222222 +irene garcia 26.297499999999996 +irene ichabod 27.540000000000003 +irene ichabod 29.34875 +irene johnson 25.418181818181814 +irene laertes 22.124285714285712 +irene laertes 22.200000000000003 +irene laertes 24.446666666666665 +irene miller 30.166666666666668 +irene nixon 18.922222222222224 +irene nixon 25.2025 +irene nixon 33.382 +irene ovid 28.256249999999998 +irene ovid 31.63 +irene ovid 32.088750000000005 +irene polk 24.63 +irene polk 25.296363636363637 +irene polk 27.137142857142855 +irene polk 30.65222222222222 +irene polk 33.760000000000005 +irene quirinius 27.044999999999998 +irene quirinius 33.07833333333334 +irene quirinius 41.865 +irene robinson 32.18875 +irene steinbeck 16.463 +irene thompson 25.281666666666666 +irene underhill 24.4025 +irene underhill 28.531 +irene van buren 25.532222222222224 +irene van buren 32.50142857142857 +irene xylophone 26.288181818181815 +jessica brown 28.968181818181822 +jessica carson 19.41 +jessica carson 24.854285714285716 +jessica carson 25.406363636363633 +jessica davidson 22.19625 +jessica davidson 23.888 +jessica davidson 26.297499999999996 +jessica davidson 26.825454545454537 +jessica ellison 22.07777777777778 +jessica ellison 33.33 +jessica falkner 22.637272727272727 +jessica garcia 14.749999999999996 +jessica garcia 29.675714285714285 +jessica ichabod 31.831249999999997 +jessica johnson 21.546666666666667 +jessica johnson 29.986363636363638 +jessica miller 28.735000000000003 +jessica nixon 19.13111111111111 +jessica nixon 26.244999999999997 +jessica ovid 25.274 +jessica ovid 33.181666666666665 +jessica polk 26.79222222222222 +jessica quirinius 20.38714285714286 +jessica quirinius 25.776000000000003 +jessica quirinius 28.26 +jessica quirinius 29.605000000000008 +jessica robinson 24.5625 +jessica thompson 25.736 +jessica thompson 30.87363636363636 +jessica underhill 16.400000000000002 +jessica underhill 25.529090909090915 +jessica underhill 31.63 +jessica van buren 24.446666666666665 +jessica white 20.812 +jessica white 23.26 +jessica white 27.807777777777776 +jessica white 29.031000000000006 +jessica white 30.654545454545453 +jessica xylophone 15.296666666666667 +jessica young 26.718333333333334 +jessica young 27.853749999999998 +jessica zipper 20.3575 +jessica zipper 24.446666666666665 +jessica zipper 29.276363636363637 +katie allen 29.029090909090915 +katie brown 24.156666666666666 +katie davidson 15.383749999999997 +katie ellison 20.978333333333335 +katie ellison 26.96 +katie falkner 24.5625 +katie garcia 27.807777777777776 +katie garcia 28.287142857142857 +katie hernandez 23.667272727272724 +katie ichabod 14.222 +katie ichabod 20.553749999999997 +katie ichabod 31.831249999999997 +katie king 20.05444444444445 +katie king 20.242222222222225 +katie king 23.342857142857145 +katie miller 26.21857142857143 +katie miller 27.675000000000004 +katie nixon 14.476999999999999 +katie ovid 28.37875 +katie polk 20.99 +katie polk 25.090000000000003 +katie robinson 30.65222222222222 +katie van buren 27.133636363636366 +katie van buren 29.675714285714285 +katie white 22.555714285714288 +katie white 24.463749999999997 +katie xylophone 25.74142857142857 +katie young 23.011250000000004 +katie young 26.650000000000002 +katie young 29.301428571428573 +katie zipper 28.26 +katie zipper 29.675714285714285 +luke allen 16.9925 +luke allen 20.595000000000002 +luke allen 27.54181818181818 +luke allen 27.887500000000003 +luke allen 33.07833333333334 +luke brown 29.34875 +luke davidson 27.3575 +luke davidson 31.473333333333333 +luke ellison 14.024444444444443 +luke ellison 22.555714285714288 +luke ellison 28.592857142857145 +luke falkner 21.855 +luke falkner 27.044999999999998 +luke garcia 27.887500000000003 +luke garcia 31.237000000000002 +luke ichabod 28.198571428571427 +luke ichabod 34.345000000000006 +luke johnson 21.239999999999995 +luke johnson 25.462727272727275 +luke johnson 30.188888888888894 +luke laertes 18.344 +luke laertes 20.817 +luke laertes 27.401999999999997 +luke laertes 30.72285714285714 +luke laertes 41.865 +luke miller 22.539000000000005 +luke ovid 16.615454545454543 +luke ovid 26.06625 +luke polk 28.163000000000004 +luke polk 28.840909090909097 +luke quirinius 27.077142857142857 +luke robinson 24.816363636363644 +luke robinson 27.110909090909093 +luke thompson 28.44454545454545 +luke underhill 22.175 +luke underhill 25.518888888888892 +luke underhill 27.34125 +luke van buren 16.54 +luke white 20.544 +luke xylophone 24.5625 +luke zipper 24.764285714285712 +mike allen 23.860000000000003 +mike brown 31.016363636363643 +mike carson 26.066363636363636 +mike carson 28.947142857142858 +mike carson 33.382 +mike davidson 20.544 +mike davidson 21.239999999999995 +mike ellison 18.922 +mike ellison 21.4175 +mike ellison 25.45272727272727 +mike ellison 27.26 +mike ellison 28.39888888888889 +mike falkner 29.397777777777776 +mike garcia 20.544 +mike garcia 24.582 +mike garcia 34.84571428571429 +mike hernandez 10.4925 +mike hernandez 17.7 +mike ichabod 26.772727272727273 +mike king 17.889999999999997 +mike king 19.294999999999998 +mike king 20.004285714285714 +mike king 23.197999999999997 +mike king 23.285 +mike king 27.401999999999997 +mike miller 31.587272727272726 +mike nixon 17.775555555555556 +mike nixon 27.044999999999998 +mike polk 22.175 +mike polk 23.751428571428573 +mike polk 28.095 +mike quirinius 19.13111111111111 +mike steinbeck 14.222 +mike steinbeck 18.100909090909088 +mike steinbeck 18.344 +mike steinbeck 33.760000000000005 +mike van buren 23.42 +mike van buren 25.828333333333333 +mike white 19.13111111111111 +mike white 25.755714285714284 +mike white 29.031000000000006 +mike white 30.516999999999996 +mike young 26.765 +mike young 27.766 +mike young 28.409090909090914 +mike zipper 17.306 +mike zipper 33.23 +mike zipper 41.865 +nick allen 19.331666666666667 +nick allen 32.106 +nick brown 27.578181818181818 +nick davidson 29.100000000000005 +nick ellison 24.764285714285712 +nick ellison 29.521666666666665 +nick falkner 22.555714285714288 +nick falkner 23.15888888888888 +nick garcia 21.546666666666667 +nick garcia 26.25090909090909 +nick garcia 30.166666666666668 +nick ichabod 21.855 +nick ichabod 23.479000000000003 +nick ichabod 29.100000000000005 +nick johnson 25.274 +nick johnson 29.994 +nick laertes 25.820909090909094 +nick miller 19.87888888888889 +nick nixon 17.082 +nick ovid 33.597777777777786 +nick polk 25.736 +nick quirinius 20.707500000000003 +nick quirinius 28.094285714285718 +nick robinson 22.396666666666665 +nick robinson 25.298749999999995 +nick steinbeck 21.192857142857143 +nick thompson 30.72285714285714 +nick underhill 29.345 +nick van buren 25.152727272727272 +nick xylophone 26.948333333333334 +nick young 23.751428571428573 +nick young 24.810000000000002 +nick zipper 24.854285714285716 +nick zipper 27.353333333333335 +oscar allen 18.815 +oscar brown 26.948333333333334 +oscar carson 24.764285714285712 +oscar carson 27.766 +oscar carson 28.094285714285718 +oscar carson 28.31555555555556 +oscar carson 35.22818181818182 +oscar davidson 17.535 +oscar ellison 22.121428571428574 +oscar ellison 28.735000000000003 +oscar falkner 19.294999999999998 +oscar garcia 20.62636363636364 +oscar hernandez 22.539000000000005 +oscar hernandez 23.31888888888889 +oscar ichabod 20.818333333333335 +oscar ichabod 21.546666666666667 +oscar ichabod 26.914545454545454 +oscar ichabod 28.811111111111106 +oscar johnson 22.381818181818183 +oscar johnson 24.266363636363643 +oscar king 15.296666666666667 +oscar king 25.580000000000002 +oscar king 28.37875 +oscar laertes 21.51818181818182 +oscar laertes 23.285 +oscar laertes 24.4025 +oscar laertes 25.345454545454547 +oscar nixon 18.88111111111111 +oscar ovid 24.854285714285716 +oscar ovid 25.274 +oscar ovid 33.29636363636364 +oscar polk 19.331666666666667 +oscar polk 29.34875 +oscar quirinius 22.928 +oscar quirinius 25.66727272727273 +oscar quirinius 25.970909090909092 +oscar quirinius 29.66666666666667 +oscar robinson 20.90666666666667 +oscar robinson 21.855 +oscar robinson 23.42 +oscar robinson 32.90500000000001 +oscar steinbeck 32.02818181818182 +oscar thompson 20.817 +oscar thompson 21.477000000000004 +oscar thompson 21.843636363636367 +oscar thompson 23.559000000000005 +oscar underhill 22.555714285714288 +oscar van buren 27.210000000000008 +oscar van buren 28.592857142857145 +oscar van buren 31.375454545454545 +oscar white 20.818333333333335 +oscar white 21.748 +oscar white 24.582 +oscar white 28.287142857142857 +oscar xylophone 25.845 +oscar xylophone 28.735000000000003 +oscar xylophone 30.72285714285714 +oscar zipper 24.511111111111113 +oscar zipper 25.067777777777778 +oscar zipper 26.21857142857143 +priscilla brown 14.222 +priscilla brown 27.044999999999998 +priscilla brown 30.19909090909091 +priscilla carson 18.07 +priscilla carson 20.70875 +priscilla carson 26.297499999999996 +priscilla ichabod 29.451111111111118 +priscilla ichabod 29.878888888888884 +priscilla johnson 16.9925 +priscilla johnson 22.050000000000004 +priscilla johnson 24.093000000000004 +priscilla johnson 29.200000000000003 +priscilla johnson 29.246 +priscilla king 15.536666666666669 +priscilla nixon 18.9 +priscilla nixon 30.516999999999996 +priscilla ovid 16.005000000000003 +priscilla ovid 29.88111111111111 +priscilla polk 28.018888888888892 +priscilla quirinius 23.064999999999998 +priscilla thompson 27.077142857142857 +priscilla underhill 28.160999999999994 +priscilla underhill 28.56571428571429 +priscilla van buren 20.7 +priscilla van buren 21.830000000000002 +priscilla van buren 24.665 +priscilla white 29.23375 +priscilla xylophone 19.331666666666667 +priscilla xylophone 23.185 +priscilla xylophone 27.34125 +priscilla young 26.32777777777778 +priscilla young 30.613999999999997 +priscilla zipper 13.498 +priscilla zipper 31.972727272727266 +quinn allen 28.786666666666672 +quinn allen 29.471818181818175 +quinn brown 26.314285714285713 +quinn brown 27.38 +quinn brown 28.39888888888889 +quinn davidson 20.808 +quinn davidson 22.71285714285714 +quinn davidson 23.306250000000002 +quinn davidson 27.34125 +quinn ellison 25.002857142857142 +quinn ellison 33.760000000000005 +quinn garcia 20.544 +quinn garcia 27.055999999999994 +quinn garcia 29.183333333333334 +quinn garcia 31.831249999999997 +quinn ichabod 22.101818181818185 +quinn king 17.535 +quinn king 18.035714285714285 +quinn laertes 24.511111111111113 +quinn laertes 28.876250000000002 +quinn laertes 29.202857142857145 +quinn nixon 21.75111111111111 +quinn ovid 29.64125 +quinn quirinius 18.922 +quinn robinson 23.985 +quinn steinbeck 27.077142857142857 +quinn steinbeck 28.160999999999994 +quinn thompson 21.99888888888889 +quinn thompson 34.46857142857143 +quinn underhill 20.113333333333333 +quinn underhill 23.107272727272726 +quinn underhill 26.224285714285717 +quinn van buren 23.612222222222222 +quinn young 24.5625 +quinn zipper 17.889999999999997 +quinn zipper 20.163333333333338 +rachel allen 28.446666666666665 +rachel allen 41.865 +rachel brown 20.92875 +rachel brown 26.21857142857143 +rachel brown 27.905 +rachel brown 30.166666666666668 +rachel brown 35.345 +rachel carson 28.735000000000003 +rachel carson 31.715999999999998 +rachel davidson 29.100000000000005 +rachel ellison 27.055454545454552 +rachel falkner 14.812499999999998 +rachel falkner 28.876250000000002 +rachel falkner 29.308888888888887 +rachel falkner 31.831249999999997 +rachel johnson 31.63 +rachel king 24.511111111111113 +rachel king 30.873749999999998 +rachel laertes 17.306 +rachel laertes 26.765 +rachel ovid 24.042727272727276 +rachel ovid 28.01181818181818 +rachel polk 21.12375 +rachel quirinius 29.831249999999997 +rachel robinson 10.4925 +rachel robinson 22.264444444444447 +rachel robinson 33.43125 +rachel thompson 16.720000000000002 +rachel thompson 26.905714285714286 +rachel thompson 28.876250000000002 +rachel underhill 21.75111111111111 +rachel white 22.175 +rachel white 31.580000000000002 +rachel young 28.150000000000002 +rachel zipper 22.187142857142856 +rachel zipper 33.760000000000005 +sarah carson 21.86818181818182 +sarah carson 22.175 +sarah carson 33.43125 +sarah ellison 17.535 +sarah falkner 29.34875 +sarah falkner 29.64125 +sarah garcia 10.4925 +sarah garcia 20.812 +sarah garcia 28.2175 +sarah ichabod 26.948333333333334 +sarah ichabod 33.62375 +sarah johnson 21.546666666666667 +sarah johnson 24.978000000000005 +sarah johnson 29.608000000000004 +sarah johnson 33.760000000000005 +sarah king 19.41 +sarah king 27.055999999999994 +sarah miller 24.815454545454543 +sarah ovid 28.31625 +sarah robinson 13.498 +sarah robinson 28.256249999999998 +sarah steinbeck 23.26 +sarah white 21.75111111111111 +sarah white 31.63 +sarah xylophone 21.964545454545455 +sarah young 29.335555555555555 +sarah zipper 29.521666666666665 +tom brown 22.873333333333335 +tom brown 30.415555555555557 +tom carson 22.4025 +tom carson 28.39888888888889 +tom carson 29.64125 +tom davidson 30.61142857142857 +tom ellison 23.568 +tom ellison 27.884999999999998 +tom ellison 32.02625 +tom falkner 13.72 +tom falkner 19.849999999999998 +tom hernandez 16.720000000000002 +tom hernandez 29.974285714285713 +tom ichabod 20.113333333333333 +tom johnson 27.077142857142857 +tom johnson 32.90500000000001 +tom king 21.855 +tom laertes 17.981666666666666 +tom laertes 21.80857142857143 +tom miller 18.922 +tom miller 21.239999999999995 +tom miller 22.396666666666665 +tom nixon 27.005000000000003 +tom ovid 34.84571428571429 +tom polk 29.521666666666665 +tom polk 29.805 +tom quirinius 24.764285714285712 +tom quirinius 36.46857142857143 +tom robinson 16.18 +tom robinson 18.07 +tom robinson 27.34125 +tom robinson 34.958571428571425 +tom steinbeck 30.613999999999997 +tom van buren 22.6475 +tom van buren 23.13555555555556 +tom van buren 24.4025 +tom white 27.715714285714284 +tom young 19.41 +tom young 24.63 +tom zipper 22.902 +ulysses brown 16.223333333333333 +ulysses carson 16.3475 +ulysses carson 22.057500000000005 +ulysses carson 28.256249999999998 +ulysses carson 28.27285714285714 +ulysses davidson 24.701249999999998 +ulysses ellison 29.52333333333333 +ulysses garcia 33.382 +ulysses hernandez 18.421818181818185 +ulysses hernandez 20.443749999999998 +ulysses hernandez 22.365 +ulysses ichabod 24.63 +ulysses ichabod 33.24333333333333 +ulysses johnson 33.43125 +ulysses king 27.083333333333332 +ulysses laertes 26.915000000000003 +ulysses laertes 27.305 +ulysses laertes 28.501111111111115 +ulysses miller 18.22 +ulysses miller 26.21857142857143 +ulysses nixon 30.65222222222222 +ulysses ovid 21.366666666666667 +ulysses polk 22.555714285714288 +ulysses polk 22.66625 +ulysses polk 25.11777777777778 +ulysses polk 27.141666666666666 +ulysses quirinius 33.07833333333334 +ulysses robinson 21.12375 +ulysses steinbeck 23.751428571428573 +ulysses steinbeck 25.931428571428572 +ulysses thompson 22.264444444444447 +ulysses underhill 20.812 +ulysses underhill 23.751428571428573 +ulysses underhill 25.071666666666662 +ulysses underhill 25.828333333333333 +ulysses underhill 25.865 +ulysses underhill 28.722499999999997 +ulysses underhill 35.268888888888895 +ulysses van buren 22.134999999999998 +ulysses white 15.296666666666667 +ulysses white 32.093333333333334 +ulysses xylophone 20.38714285714286 +ulysses xylophone 25.274 +ulysses xylophone 29.64125 +ulysses young 22.213333333333335 +ulysses young 22.90285714285714 +ulysses young 32.93125 +victor allen 24.82875 +victor allen 27.51 +victor brown 21.621250000000003 +victor brown 23.73 +victor brown 26.21857142857143 +victor brown 27.548571428571428 +victor davidson 22.391666666666666 +victor davidson 33.16428571428572 +victor davidson 35.197500000000005 +victor ellison 11.100000000000001 +victor ellison 30.96857142857143 +victor hernandez 10.4925 +victor hernandez 18.922 +victor hernandez 24.301250000000003 +victor hernandez 26.69857142857143 +victor hernandez 35.358333333333334 +victor johnson 16.580000000000002 +victor johnson 27.516666666666666 +victor johnson 32.106 +victor king 19.962857142857143 +victor king 33.01857142857143 +victor laertes 21.78142857142857 +victor laertes 33.10999999999999 +victor miller 21.93285714285714 +victor nixon 20.419999999999998 +victor nixon 33.69 +victor ovid 28.75857142857143 +victor polk 18.43111111111111 +victor quirinius 17.84777777777778 +victor quirinius 27.53 +victor robinson 19.37 +victor robinson 20.38714285714286 +victor steinbeck 20.818333333333335 +victor steinbeck 25.16 +victor steinbeck 30.503749999999997 +victor thompson 23.987142857142857 +victor van buren 27.009999999999998 +victor van buren 33.43125 +victor white 24.322857142857142 +victor white 28.287142857142857 +victor xylophone 11.807142857142859 +victor xylophone 13.988571428571428 +victor xylophone 16.720000000000002 +victor xylophone 19.686666666666667 +victor xylophone 37.20428571428572 +victor young 22.264444444444447 +victor zipper 24.854285714285716 +wendy allen 28.24142857142857 +wendy allen 29.675714285714285 +wendy allen 34.275 +wendy brown 22.482857142857142 +wendy brown 27.79714285714286 +wendy ellison 16.80888888888889 +wendy ellison 18.135 +wendy falkner 22.628888888888884 +wendy falkner 23.325000000000003 +wendy falkner 24.0375 +wendy garcia 19.307142857142853 +wendy garcia 21.761428571428574 +wendy garcia 24.63 +wendy garcia 24.854285714285716 +wendy hernandez 16.60875 +wendy ichabod 28.26 +wendy king 22.5 +wendy king 24.793333333333333 +wendy king 28.252857142857145 +wendy laertes 25.881428571428568 +wendy laertes 30.338333333333328 +wendy laertes 30.52857142857143 +wendy miller 15.478333333333332 +wendy miller 25.34333333333333 +wendy nixon 19.54714285714286 +wendy nixon 27.003333333333334 +wendy ovid 14.283750000000001 +wendy ovid 30.878333333333334 +wendy polk 21.69375 +wendy polk 24.63 +wendy quirinius 28.731428571428573 +wendy quirinius 29.74333333333333 +wendy robinson 16.720000000000002 +wendy robinson 23.834285714285716 +wendy robinson 29.911666666666672 +wendy steinbeck 29.272857142857145 +wendy thompson 18.17875 +wendy thompson 22.544285714285714 +wendy underhill 21.69625 +wendy underhill 27.077142857142857 +wendy underhill 30.03333333333333 +wendy van buren 28.624285714285715 +wendy van buren 29.28333333333333 +wendy white 24.4025 +wendy xylophone 16.84 +wendy xylophone 23.426666666666666 +wendy young 20.80125 +wendy young 32.693333333333335 +xavier allen 16.535 +xavier allen 17.398333333333337 +xavier allen 35.708333333333336 +xavier brown 20.787142857142857 +xavier brown 24.764285714285712 +xavier brown 31.784999999999997 +xavier carson 20.818333333333335 +xavier carson 32.106 +xavier davidson 16.862857142857145 +xavier davidson 20.53625 +xavier davidson 27.353333333333335 +xavier ellison 17.991666666666667 +xavier ellison 23.976666666666663 +xavier garcia 35.84428571428572 +xavier hernandez 22.654285714285713 +xavier hernandez 26.948333333333334 +xavier hernandez 28.075 +xavier ichabod 20.344285714285714 +xavier ichabod 20.818333333333335 +xavier johnson 15.754285714285714 +xavier johnson 19.490000000000002 +xavier king 29.246666666666666 +xavier king 29.521666666666665 +xavier laertes 19.294999999999998 +xavier ovid 28.51 +xavier polk 12.728333333333333 +xavier polk 19.37 +xavier polk 22.548333333333332 +xavier polk 28.465714285714284 +xavier quirinius 9.991428571428571 +xavier quirinius 24.156666666666666 +xavier quirinius 25.69666666666667 +xavier quirinius 25.828333333333333 +xavier thompson 23.961428571428574 +xavier underhill 21.830000000000002 +xavier white 19.331666666666667 +xavier white 35.345 +xavier xylophone 21.187142857142856 +xavier zipper 17.488333333333333 +yuri allen 15.705714285714285 +yuri allen 20.808 +yuri brown 19.53 +yuri brown 22.457142857142856 +yuri carson 25.699999999999996 +yuri carson 27.216666666666665 +yuri ellison 15.034999999999998 +yuri ellison 28.463333333333335 +yuri falkner 17.81833333333333 +yuri falkner 19.294999999999998 +yuri garcia 28.287142857142857 +yuri hernandez 32.395 +yuri johnson 25.828333333333333 +yuri johnson 27.301666666666666 +yuri johnson 29.578333333333337 +yuri king 19.921666666666663 +yuri laertes 16.18 +yuri laertes 30.519999999999996 +yuri nixon 16.383333333333333 +yuri nixon 25.828333333333333 +yuri polk 16.18 +yuri polk 20.503333333333334 +yuri polk 30.16333333333333 +yuri quirinius 20.311666666666667 +yuri quirinius 23.185 +yuri quirinius 24.828333333333333 +yuri steinbeck 19.331666666666667 +yuri steinbeck 28.50666666666667 +yuri thompson 35.27 +yuri underhill 23.042857142857144 +yuri underhill 28.786666666666665 +yuri white 30.72285714285714 +yuri xylophone 24.173333333333332 +zach allen 8.983333333333333 +zach brown 18.922 +zach brown 23.036666666666665 +zach brown 29.72666666666667 +zach brown 31.58285714285714 +zach brown 33.07833333333334 +zach carson 27.110000000000003 +zach ellison 18.168333333333333 +zach falkner 16.18 +zach falkner 30.83285714285714 +zach garcia 16.586666666666666 +zach garcia 22.53333333333333 +zach garcia 28.13166666666667 +zach garcia 34.84571428571429 +zach ichabod 17.535 +zach ichabod 30.72285714285714 +zach king 19.878333333333334 +zach king 25.643333333333334 +zach king 28.646666666666665 +zach miller 23.285 +zach miller 23.366666666666664 +zach miller 30.46833333333333 +zach ovid 23.94666666666667 +zach ovid 28.75166666666667 +zach ovid 28.763333333333335 +zach ovid 34.84571428571429 +zach quirinius 20.755 +zach robinson 21.546666666666667 +zach steinbeck 27.243333333333336 +zach steinbeck 30.073333333333334 +zach thompson 14.222 +zach thompson 24.755 +zach underhill 31.885 +zach white 20.208333333333332 +zach xylophone 10.485 +zach xylophone 20.113333333333333 +zach young 20.176666666666666 +zach zipper 21.709999999999997 +zach zipper 22.264999999999997 +zach zipper 34.01166666666667 diff --git a/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-5-ee44c5cdc80e1c832b702f9fb76d8145 b/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-5-ee44c5cdc80e1c832b702f9fb76d8145 new file mode 100644 index 0000000000000..a9ae190825a01 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-5-ee44c5cdc80e1c832b702f9fb76d8145 @@ -0,0 +1,1049 @@ + 65560 + 65718 + 65740 +alice allen 65662 +alice allen 65720 +alice allen 65758 +alice brown 65696 +alice carson 65559 +alice davidson 65547 +alice falkner 65669 +alice garcia 65613 +alice hernandez 65737 +alice hernandez 65784 +alice johnson 65739 +alice king 65660 +alice king 65738 +alice king 131281 +alice laertes 65669 +alice laertes 65671 +alice miller 65590 +alice nixon 65586 +alice nixon 65595 +alice nixon 65604 +alice ovid 65737 +alice polk 65548 +alice quirinius 65636 +alice quirinius 65728 +alice robinson 65606 +alice robinson 65789 +alice steinbeck 65578 +alice steinbeck 65673 +alice steinbeck 65786 +alice underhill 65750 +alice van buren 65562 +alice xylophone 65585 +alice xylophone 65599 +alice xylophone 131247 +alice zipper 65553 +alice zipper 65662 +alice zipper 65766 +bob brown 65584 +bob brown 65777 +bob brown 65783 +bob carson 65713 +bob davidson 65664 +bob davidson 65693 +bob davidson 65768 +bob ellison 65591 +bob ellison 65624 +bob ellison 65721 +bob ellison 65760 +bob falkner 65789 +bob garcia 65585 +bob garcia 65598 +bob garcia 65673 +bob garcia 65754 +bob garcia 65782 +bob hernandez 131340 +bob ichabod 65549 +bob king 65715 +bob king 65757 +bob king 65783 +bob laertes 65602 +bob laertes 131447 +bob miller 65608 +bob ovid 65564 +bob ovid 65686 +bob ovid 196959 +bob ovid 196973 +bob polk 65594 +bob quirinius 65700 +bob steinbeck 65637 +bob van buren 65778 +bob white 65543 +bob white 65605 +bob xylophone 65574 +bob xylophone 65666 +bob young 65556 +bob zipper 65559 +bob zipper 65633 +bob zipper 65739 +calvin allen 65669 +calvin brown 65537 +calvin brown 131272 +calvin brown 197027 +calvin carson 65637 +calvin davidson 65541 +calvin davidson 65564 +calvin ellison 65667 +calvin falkner 65573 +calvin falkner 65596 +calvin falkner 65778 +calvin falkner 131397 +calvin falkner 131411 +calvin falkner 131433 +calvin garcia 131212 +calvin hernandez 131251 +calvin johnson 65731 +calvin laertes 65570 +calvin laertes 65684 +calvin nixon 65654 +calvin nixon 131386 +calvin nixon 131503 +calvin ovid 65554 +calvin ovid 65643 +calvin ovid 65715 +calvin ovid 196944 +calvin polk 65731 +calvin quirinius 65741 +calvin quirinius 65769 +calvin robinson 131320 +calvin steinbeck 131271 +calvin steinbeck 131326 +calvin steinbeck 131415 +calvin thompson 65560 +calvin thompson 131244 +calvin underhill 196944 +calvin van buren 65771 +calvin van buren 131138 +calvin white 65553 +calvin white 65561 +calvin xylophone 65575 +calvin xylophone 65596 +calvin xylophone 262686 +calvin young 65746 +calvin young 131168 +calvin zipper 65669 +calvin zipper 131476 +david allen 65588 +david allen 131222 +david brown 65637 +david brown 131303 +david davidson 65756 +david davidson 65778 +david davidson 65779 +david davidson 131342 +david ellison 65724 +david ellison 65724 +david ellison 131224 +david hernandez 197083 +david ichabod 131454 +david ichabod 197085 +david laertes 65762 +david nixon 65536 +david ovid 65623 +david ovid 196766 +david quirinius 65759 +david quirinius 65779 +david quirinius 131303 +david robinson 65762 +david robinson 65775 +david thompson 65550 +david underhill 65662 +david underhill 65751 +david underhill 131198 +david van buren 65634 +david van buren 262584 +david white 65678 +david xylophone 65537 +david xylophone 131426 +david xylophone 131447 +david young 65551 +david young 131255 +ethan allen 131460 +ethan brown 65539 +ethan brown 65617 +ethan brown 65685 +ethan brown 65685 +ethan brown 65722 +ethan brown 131483 +ethan carson 197189 +ethan ellison 65714 +ethan ellison 131302 +ethan falkner 131222 +ethan falkner 131333 +ethan garcia 131507 +ethan hernandez 65618 +ethan johnson 65536 +ethan king 131280 +ethan laertes 65562 +ethan laertes 65597 +ethan laertes 65628 +ethan laertes 65680 +ethan laertes 65760 +ethan laertes 131304 +ethan laertes 328329 +ethan miller 328296 +ethan nixon 65766 +ethan ovid 65697 +ethan polk 65589 +ethan polk 65615 +ethan polk 131206 +ethan polk 197082 +ethan quirinius 65591 +ethan quirinius 196912 +ethan quirinius 196957 +ethan robinson 65547 +ethan robinson 65659 +ethan underhill 65570 +ethan van buren 131252 +ethan white 65677 +ethan white 197039 +ethan xylophone 65595 +ethan zipper 65593 +ethan zipper 131365 +fred davidson 65595 +fred davidson 65721 +fred davidson 131221 +fred ellison 65548 +fred ellison 65691 +fred ellison 65771 +fred falkner 65637 +fred falkner 131474 +fred falkner 196920 +fred hernandez 131226 +fred ichabod 131109 +fred ichabod 131520 +fred johnson 131332 +fred king 65694 +fred king 197016 +fred laertes 131354 +fred miller 65536 +fred nixon 65560 +fred nixon 65612 +fred nixon 65705 +fred nixon 196929 +fred polk 65656 +fred polk 131231 +fred polk 262645 +fred polk 262733 +fred quirinius 131486 +fred quirinius 196950 +fred robinson 65623 +fred steinbeck 65544 +fred steinbeck 65755 +fred steinbeck 131253 +fred underhill 131188 +fred van buren 65561 +fred van buren 65745 +fred van buren 131380 +fred van buren 328270 +fred white 131136 +fred young 65594 +fred young 131551 +fred zipper 196885 +gabriella allen 65677 +gabriella allen 131283 +gabriella brown 65753 +gabriella brown 197180 +gabriella carson 65586 +gabriella davidson 65565 +gabriella ellison 65706 +gabriella ellison 131505 +gabriella falkner 65767 +gabriella falkner 131183 +gabriella falkner 131397 +gabriella garcia 131127 +gabriella hernandez 131304 +gabriella hernandez 131304 +gabriella ichabod 65559 +gabriella ichabod 65712 +gabriella ichabod 131297 +gabriella ichabod 131311 +gabriella ichabod 131460 +gabriella king 65657 +gabriella king 197031 +gabriella laertes 131543 +gabriella miller 131300 +gabriella ovid 65556 +gabriella ovid 131260 +gabriella polk 65790 +gabriella polk 131425 +gabriella steinbeck 65582 +gabriella steinbeck 131248 +gabriella thompson 131528 +gabriella thompson 197181 +gabriella thompson 262632 +gabriella van buren 65644 +gabriella van buren 131238 +gabriella white 65638 +gabriella young 65699 +gabriella young 65774 +gabriella zipper 65754 +gabriella zipper 196762 +holly allen 65596 +holly brown 131315 +holly brown 131368 +holly falkner 65720 +holly hernandez 65602 +holly hernandez 65686 +holly hernandez 131387 +holly hernandez 131554 +holly ichabod 65752 +holly ichabod 131308 +holly ichabod 131473 +holly johnson 65755 +holly johnson 131240 +holly johnson 131277 +holly king 131286 +holly king 131303 +holly laertes 196950 +holly miller 131381 +holly nixon 196941 +holly nixon 328184 +holly polk 197132 +holly polk 262782 +holly robinson 131241 +holly thompson 65578 +holly thompson 65713 +holly thompson 197092 +holly underhill 65654 +holly underhill 131323 +holly underhill 131385 +holly underhill 131504 +holly van buren 131449 +holly white 131092 +holly white 262734 +holly xylophone 196792 +holly young 65765 +holly young 131229 +holly zipper 131151 +holly zipper 131545 +irene allen 131109 +irene brown 65765 +irene brown 131368 +irene brown 393929 +irene carson 262770 +irene ellison 196956 +irene ellison 196982 +irene falkner 131287 +irene falkner 197046 +irene garcia 65660 +irene garcia 131286 +irene garcia 131375 +irene ichabod 65645 +irene ichabod 131442 +irene johnson 131179 +irene laertes 131324 +irene laertes 131381 +irene laertes 131407 +irene miller 262822 +irene nixon 197105 +irene nixon 262409 +irene nixon 262565 +irene ovid 65734 +irene ovid 196935 +irene ovid 262836 +irene polk 65551 +irene polk 131189 +irene polk 131189 +irene polk 196943 +irene polk 328365 +irene quirinius 131369 +irene quirinius 196998 +irene quirinius 262855 +irene robinson 131259 +irene steinbeck 65683 +irene thompson 262719 +irene underhill 131291 +irene underhill 131386 +irene van buren 131216 +irene van buren 262539 +irene xylophone 131348 +jessica brown 393772 +jessica carson 65747 +jessica carson 131207 +jessica carson 131232 +jessica davidson 65606 +jessica davidson 65675 +jessica davidson 196917 +jessica davidson 197030 +jessica ellison 131108 +jessica ellison 196885 +jessica falkner 131270 +jessica garcia 197059 +jessica garcia 328458 +jessica ichabod 197028 +jessica johnson 131177 +jessica johnson 197085 +jessica miller 197024 +jessica nixon 131549 +jessica nixon 196682 +jessica ovid 65751 +jessica ovid 196890 +jessica polk 459409 +jessica quirinius 131222 +jessica quirinius 131248 +jessica quirinius 131294 +jessica quirinius 393878 +jessica robinson 131174 +jessica thompson 131336 +jessica thompson 196927 +jessica underhill 131218 +jessica underhill 131267 +jessica underhill 197086 +jessica van buren 65615 +jessica white 65544 +jessica white 65594 +jessica white 197012 +jessica white 262435 +jessica white 262571 +jessica xylophone 196866 +jessica young 65711 +jessica young 131183 +jessica zipper 196897 +jessica zipper 262523 +jessica zipper 262695 +katie allen 196740 +katie brown 328113 +katie davidson 131371 +katie ellison 131248 +katie ellison 197182 +katie falkner 131441 +katie garcia 131384 +katie garcia 197051 +katie hernandez 131296 +katie ichabod 131495 +katie ichabod 197131 +katie ichabod 197275 +katie king 131252 +katie king 262588 +katie king 262861 +katie miller 65661 +katie miller 262723 +katie nixon 65669 +katie ovid 65681 +katie polk 65784 +katie polk 197249 +katie robinson 131251 +katie van buren 131237 +katie van buren 197141 +katie white 262510 +katie white 262860 +katie xylophone 197034 +katie young 65644 +katie young 328173 +katie young 393859 +katie zipper 65733 +katie zipper 328287 +luke allen 65776 +luke allen 131268 +luke allen 196819 +luke allen 196855 +luke allen 328011 +luke brown 196967 +luke davidson 65656 +luke davidson 131573 +luke ellison 65582 +luke ellison 131343 +luke ellison 197118 +luke falkner 196797 +luke falkner 196837 +luke garcia 65778 +luke garcia 393974 +luke ichabod 65629 +luke ichabod 262574 +luke johnson 131302 +luke johnson 131312 +luke johnson 131361 +luke laertes 131226 +luke laertes 131504 +luke laertes 197018 +luke laertes 197153 +luke laertes 197177 +luke miller 197052 +luke ovid 65569 +luke ovid 262745 +luke polk 65658 +luke polk 262627 +luke quirinius 131233 +luke robinson 65634 +luke robinson 262569 +luke thompson 196858 +luke underhill 65651 +luke underhill 131240 +luke underhill 328248 +luke van buren 131398 +luke white 65693 +luke xylophone 131312 +luke zipper 131297 +mike allen 196928 +mike brown 197149 +mike carson 65751 +mike carson 131284 +mike carson 393711 +mike davidson 196917 +mike davidson 262912 +mike ellison 65598 +mike ellison 131366 +mike ellison 131412 +mike ellison 131509 +mike ellison 262704 +mike falkner 328183 +mike garcia 131530 +mike garcia 328305 +mike garcia 328461 +mike hernandez 131301 +mike hernandez 328384 +mike ichabod 131157 +mike king 196965 +mike king 197091 +mike king 197121 +mike king 262471 +mike king 262527 +mike king 328279 +mike miller 131317 +mike nixon 131328 +mike nixon 262653 +mike polk 131240 +mike polk 196899 +mike polk 262885 +mike quirinius 525126 +mike steinbeck 65550 +mike steinbeck 131201 +mike steinbeck 131490 +mike steinbeck 262490 +mike van buren 131548 +mike van buren 262547 +mike white 197000 +mike white 197060 +mike white 262425 +mike white 328482 +mike young 196935 +mike young 196976 +mike young 328084 +mike zipper 131147 +mike zipper 197075 +mike zipper 328517 +nick allen 131192 +nick allen 197024 +nick brown 131503 +nick davidson 262686 +nick ellison 197119 +nick ellison 197119 +nick falkner 65583 +nick falkner 328561 +nick garcia 131318 +nick garcia 262755 +nick garcia 328281 +nick ichabod 131430 +nick ichabod 196812 +nick ichabod 328593 +nick johnson 131453 +nick johnson 262597 +nick laertes 196732 +nick miller 131490 +nick nixon 262547 +nick ovid 328266 +nick polk 196852 +nick quirinius 131438 +nick quirinius 328176 +nick robinson 131326 +nick robinson 196980 +nick steinbeck 131250 +nick thompson 65610 +nick underhill 65619 +nick van buren 196795 +nick xylophone 196972 +nick young 394136 +nick young 459634 +nick zipper 262954 +nick zipper 394218 +oscar allen 262674 +oscar brown 196916 +oscar carson 131099 +oscar carson 131330 +oscar carson 196731 +oscar carson 196733 +oscar carson 196918 +oscar davidson 262554 +oscar ellison 65630 +oscar ellison 197116 +oscar falkner 197145 +oscar garcia 328305 +oscar hernandez 197022 +oscar hernandez 328315 +oscar ichabod 131302 +oscar ichabod 131309 +oscar ichabod 196760 +oscar ichabod 196874 +oscar johnson 196942 +oscar johnson 197203 +oscar king 196793 +oscar king 196944 +oscar king 328236 +oscar laertes 131208 +oscar laertes 262522 +oscar laertes 262842 +oscar laertes 328364 +oscar nixon 65596 +oscar ovid 131228 +oscar ovid 262580 +oscar ovid 393817 +oscar polk 131078 +oscar polk 131260 +oscar quirinius 131103 +oscar quirinius 196748 +oscar quirinius 196829 +oscar quirinius 262838 +oscar robinson 196874 +oscar robinson 262803 +oscar robinson 393773 +oscar robinson 394087 +oscar steinbeck 328432 +oscar thompson 196826 +oscar thompson 196992 +oscar thompson 262593 +oscar thompson 459401 +oscar underhill 131301 +oscar van buren 131134 +oscar van buren 328162 +oscar van buren 394034 +oscar white 131457 +oscar white 262345 +oscar white 328538 +oscar white 459337 +oscar xylophone 65773 +oscar xylophone 262708 +oscar xylophone 262906 +oscar zipper 196904 +oscar zipper 262512 +oscar zipper 328262 +priscilla brown 196950 +priscilla brown 328237 +priscilla brown 328624 +priscilla carson 262488 +priscilla carson 262510 +priscilla carson 262703 +priscilla ichabod 131178 +priscilla ichabod 131303 +priscilla johnson 131224 +priscilla johnson 196906 +priscilla johnson 196994 +priscilla johnson 197184 +priscilla johnson 394171 +priscilla king 262692 +priscilla nixon 262691 +priscilla nixon 394188 +priscilla ovid 65541 +priscilla ovid 197067 +priscilla polk 394009 +priscilla quirinius 131306 +priscilla thompson 196875 +priscilla underhill 197084 +priscilla underhill 262701 +priscilla van buren 65685 +priscilla van buren 131368 +priscilla van buren 196814 +priscilla white 196893 +priscilla xylophone 131473 +priscilla xylophone 262597 +priscilla xylophone 262785 +priscilla young 131392 +priscilla young 262788 +priscilla zipper 393888 +priscilla zipper 394031 +quinn allen 197095 +quinn allen 394225 +quinn brown 131470 +quinn brown 131473 +quinn brown 262642 +quinn davidson 197079 +quinn davidson 197112 +quinn davidson 262510 +quinn davidson 459427 +quinn ellison 197268 +quinn ellison 328130 +quinn garcia 65604 +quinn garcia 131321 +quinn garcia 197067 +quinn garcia 328528 +quinn ichabod 65564 +quinn king 65649 +quinn king 196879 +quinn laertes 65542 +quinn laertes 196877 +quinn laertes 262466 +quinn nixon 196837 +quinn ovid 525126 +quinn quirinius 328235 +quinn robinson 131378 +quinn steinbeck 131484 +quinn steinbeck 262528 +quinn thompson 197030 +quinn thompson 262717 +quinn underhill 262791 +quinn underhill 328146 +quinn underhill 393824 +quinn van buren 197234 +quinn young 65647 +quinn zipper 131466 +quinn zipper 262658 +rachel allen 65661 +rachel allen 196935 +rachel brown 131220 +rachel brown 328076 +rachel brown 328320 +rachel brown 393835 +rachel brown 524988 +rachel carson 131259 +rachel carson 459393 +rachel davidson 262632 +rachel ellison 393845 +rachel falkner 196947 +rachel falkner 262474 +rachel falkner 394046 +rachel falkner 525086 +rachel johnson 65658 +rachel king 131354 +rachel king 196907 +rachel laertes 131391 +rachel laertes 197105 +rachel ovid 262664 +rachel ovid 328195 +rachel polk 328389 +rachel quirinius 262779 +rachel robinson 262491 +rachel robinson 262862 +rachel robinson 590712 +rachel thompson 197034 +rachel thompson 328158 +rachel thompson 394094 +rachel underhill 197033 +rachel white 131399 +rachel white 197190 +rachel young 196967 +rachel zipper 328223 +rachel zipper 394149 +sarah carson 131379 +sarah carson 196870 +sarah carson 262491 +sarah ellison 197095 +sarah falkner 131262 +sarah falkner 328251 +sarah garcia 196963 +sarah garcia 197030 +sarah garcia 459657 +sarah ichabod 262504 +sarah ichabod 262766 +sarah johnson 131409 +sarah johnson 262783 +sarah johnson 328591 +sarah johnson 394043 +sarah king 196998 +sarah king 328416 +sarah miller 196893 +sarah ovid 131199 +sarah robinson 262868 +sarah robinson 394066 +sarah steinbeck 262650 +sarah white 197059 +sarah white 262579 +sarah xylophone 131336 +sarah young 394123 +sarah zipper 262818 +tom brown 196848 +tom brown 328268 +tom carson 197328 +tom carson 262517 +tom carson 656251 +tom davidson 262864 +tom ellison 196974 +tom ellison 328416 +tom ellison 393921 +tom falkner 393809 +tom falkner 459407 +tom hernandez 262525 +tom hernandez 328085 +tom ichabod 197048 +tom johnson 328321 +tom johnson 393865 +tom king 196951 +tom laertes 262657 +tom laertes 459805 +tom miller 131278 +tom miller 131459 +tom miller 262633 +tom nixon 262588 +tom ovid 262595 +tom polk 328470 +tom polk 328584 +tom quirinius 262597 +tom quirinius 262681 +tom robinson 196978 +tom robinson 328481 +tom robinson 459857 +tom robinson 525095 +tom steinbeck 262426 +tom van buren 131389 +tom van buren 328095 +tom van buren 328313 +tom white 328128 +tom young 131080 +tom young 393692 +tom zipper 197167 +ulysses brown 196815 +ulysses carson 131277 +ulysses carson 262450 +ulysses carson 262937 +ulysses carson 328311 +ulysses davidson 262750 +ulysses ellison 262445 +ulysses garcia 328445 +ulysses hernandez 131414 +ulysses hernandez 196871 +ulysses hernandez 394370 +ulysses ichabod 393834 +ulysses ichabod 459582 +ulysses johnson 262966 +ulysses king 131363 +ulysses laertes 262739 +ulysses laertes 328412 +ulysses laertes 328462 +ulysses miller 262661 +ulysses miller 328360 +ulysses nixon 394194 +ulysses ovid 328289 +ulysses polk 65563 +ulysses polk 197046 +ulysses polk 328294 +ulysses polk 590698 +ulysses quirinius 525643 +ulysses robinson 394160 +ulysses steinbeck 196783 +ulysses steinbeck 262778 +ulysses thompson 262607 +ulysses underhill 131214 +ulysses underhill 196937 +ulysses underhill 197027 +ulysses underhill 262623 +ulysses underhill 262623 +ulysses underhill 262648 +ulysses underhill 262836 +ulysses van buren 196944 +ulysses white 197033 +ulysses white 393988 +ulysses xylophone 262695 +ulysses xylophone 328151 +ulysses xylophone 328747 +ulysses young 196903 +ulysses young 394037 +ulysses young 459782 +victor allen 197189 +victor allen 262651 +victor brown 262544 +victor brown 262799 +victor brown 327900 +victor brown 591265 +victor davidson 197173 +victor davidson 262486 +victor davidson 328274 +victor ellison 328618 +victor ellison 393962 +victor hernandez 197041 +victor hernandez 197132 +victor hernandez 262771 +victor hernandez 328261 +victor hernandez 459902 +victor johnson 131155 +victor johnson 131169 +victor johnson 394168 +victor king 131486 +victor king 328509 +victor laertes 262573 +victor laertes 328435 +victor miller 196784 +victor nixon 196987 +victor nixon 394249 +victor ovid 196882 +victor polk 262462 +victor quirinius 65620 +victor quirinius 328301 +victor robinson 328334 +victor robinson 394031 +victor steinbeck 65661 +victor steinbeck 262560 +victor steinbeck 262750 +victor thompson 65548 +victor van buren 197173 +victor van buren 328261 +victor white 262588 +victor white 328039 +victor xylophone 131203 +victor xylophone 262596 +victor xylophone 328191 +victor xylophone 393913 +victor xylophone 459542 +victor young 131258 +victor zipper 131349 +wendy allen 131402 +wendy allen 196954 +wendy allen 328359 +wendy brown 328365 +wendy brown 459501 +wendy ellison 262718 +wendy ellison 328191 +wendy falkner 197009 +wendy falkner 262430 +wendy falkner 328177 +wendy garcia 65746 +wendy garcia 393974 +wendy garcia 459883 +wendy garcia 459926 +wendy hernandez 65650 +wendy ichabod 262665 +wendy king 262545 +wendy king 328229 +wendy king 393951 +wendy laertes 262739 +wendy laertes 262794 +wendy laertes 328315 +wendy miller 131377 +wendy miller 328161 +wendy nixon 131258 +wendy nixon 196893 +wendy ovid 196952 +wendy ovid 459594 +wendy polk 328520 +wendy polk 394310 +wendy quirinius 328703 +wendy quirinius 394360 +wendy robinson 131316 +wendy robinson 394030 +wendy robinson 459665 +wendy steinbeck 262645 +wendy thompson 262725 +wendy thompson 393865 +wendy underhill 328445 +wendy underhill 394295 +wendy underhill 460068 +wendy van buren 65699 +wendy van buren 196964 +wendy white 328135 +wendy xylophone 262894 +wendy xylophone 525344 +wendy young 197017 +wendy young 721936 +xavier allen 197025 +xavier allen 525393 +xavier allen 525839 +xavier brown 197058 +xavier brown 262626 +xavier brown 328388 +xavier carson 196990 +xavier carson 328415 +xavier davidson 65644 +xavier davidson 262745 +xavier davidson 393825 +xavier ellison 197095 +xavier ellison 328447 +xavier garcia 262590 +xavier hernandez 196847 +xavier hernandez 197077 +xavier hernandez 393838 +xavier ichabod 262600 +xavier ichabod 328157 +xavier johnson 197084 +xavier johnson 262785 +xavier king 196919 +xavier king 262774 +xavier laertes 262770 +xavier ovid 328414 +xavier polk 196844 +xavier polk 328474 +xavier polk 394013 +xavier polk 590931 +xavier quirinius 65650 +xavier quirinius 131140 +xavier quirinius 328382 +xavier quirinius 459669 +xavier thompson 393799 +xavier underhill 197012 +xavier white 196858 +xavier white 262712 +xavier xylophone 131250 +xavier zipper 394070 +yuri allen 131129 +yuri allen 459977 +yuri brown 262640 +yuri brown 393858 +yuri carson 459799 +yuri carson 591063 +yuri ellison 197085 +yuri ellison 459558 +yuri falkner 196857 +yuri falkner 525350 +yuri garcia 328378 +yuri hernandez 262588 +yuri johnson 393861 +yuri johnson 394444 +yuri johnson 525638 +yuri king 525526 +yuri laertes 131551 +yuri laertes 459611 +yuri nixon 262644 +yuri nixon 393936 +yuri polk 328197 +yuri polk 328404 +yuri polk 328481 +yuri quirinius 131092 +yuri quirinius 196898 +yuri quirinius 525159 +yuri steinbeck 394037 +yuri steinbeck 525180 +yuri thompson 459710 +yuri underhill 328325 +yuri underhill 459781 +yuri white 131252 +yuri xylophone 262809 +zach allen 394026 +zach brown 262789 +zach brown 262789 +zach brown 459521 +zach brown 459846 +zach brown 590938 +zach carson 262320 +zach ellison 262757 +zach falkner 262608 +zach falkner 262608 +zach garcia 262818 +zach garcia 328314 +zach garcia 393686 +zach garcia 394011 +zach ichabod 262518 +zach ichabod 262563 +zach king 196780 +zach king 196905 +zach king 459991 +zach miller 196923 +zach miller 393813 +zach miller 393892 +zach ovid 196876 +zach ovid 262643 +zach ovid 328023 +zach ovid 459615 +zach quirinius 262471 +zach robinson 196967 +zach steinbeck 131394 +zach steinbeck 459294 +zach thompson 131340 +zach thompson 525538 +zach underhill 131304 +zach white 65733 +zach xylophone 262810 +zach xylophone 459455 +zach young 393615 +zach zipper 197130 +zach zipper 262496 +zach zipper 393937 diff --git a/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-6-4d78f7b1d172d20c91f5867bc13a42a0 b/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-6-4d78f7b1d172d20c91f5867bc13a42a0 new file mode 100644 index 0000000000000..b3f08818f491a --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-6-4d78f7b1d172d20c91f5867bc13a42a0 @@ -0,0 +1,1049 @@ +0.08 0.07999999821186066 +0.1 0.10000000149011612 +0.13 0.12999999523162842 +0.15 0.15000000596046448 +0.27 0.27000001072883606 +0.28 0.2800000011920929 +0.43 0.4300000071525574 +0.52 0.5199999809265137 +0.56 0.5600000023841858 +0.6 0.6000000238418579 +0.61 0.6100000143051147 +0.79 0.7900000214576721 +0.84 0.8399999737739563 +0.98 0.9800000190734863 +1.02 1.2899999916553497 +1.08 1.0800000429153442 +1.08 1.0800000429153442 +1.12 1.1200000047683716 +1.21 2.0000000596046448 +1.25 1.25 +1.27 1.2699999809265137 +1.29 1.2899999618530273 +1.31 1.309999942779541 +1.58 1.5800000429153442 +1.87 1.8700000047683716 +1.91 1.909999966621399 +1.92 3.1699999570846558 +2.07 2.069999933242798 +2.18 2.180000066757202 +2.2 2.200000047683716 +2.35 2.3499999046325684 +2.6 2.5999999046325684 +2.79 2.7899999618530273 +2.92 2.9200000762939453 +2.96 2.9600000381469727 +2.96 2.9600000381469727 +2.97 2.9700000286102295 +3.0 3.0 +3.21 3.340000033378601 +3.28 4.399999976158142 +3.33 3.3299999237060547 +3.61 3.609999895095825 +3.62 3.619999885559082 +3.82 3.819999933242798 +3.86 3.859999895095825 +3.96 3.9600000381469727 +3.97 3.9700000286102295 +4.17 7.7799999713897705 +4.32 4.320000171661377 +4.35 4.349999904632568 +4.41 4.409999847412109 +4.46 4.460000038146973 +4.47 4.46999979019165 +4.57 4.570000171661377 +4.59 4.590000152587891 +4.71 4.710000038146973 +4.72 4.71999979019165 +4.79 4.789999961853027 +4.8 4.800000190734863 +4.92 4.920000076293945 +5.08 5.079999923706055 +5.24 9.559999942779541 +5.28 5.28000020980835 +5.4 5.400000095367432 +5.44 5.440000057220459 +5.45 5.449999809265137 +5.51 5.510000228881836 +5.54 5.539999961853027 +5.62 5.619999885559082 +5.67 5.670000076293945 +5.85 5.849999904632568 +5.88 5.880000114440918 +6.29 6.289999961853027 +6.55 6.550000190734863 +6.57 11.160000324249268 +6.63 9.59000015258789 +6.67 6.670000076293945 +6.72 6.71999979019165 +6.74 6.739999771118164 +6.84 6.840000152587891 +6.87 6.869999885559082 +7.05 7.050000190734863 +7.06 11.769999980926514 +7.11 7.110000133514404 +7.54 7.539999961853027 +7.56 7.559999942779541 +7.79 7.789999961853027 +7.82 7.820000171661377 +7.96 7.960000038146973 +7.96 7.960000038146973 +7.98 7.980000019073486 +8.07 8.069999694824219 +8.07 8.069999694824219 +8.32 8.319999694824219 +8.37 11.339999914169312 +8.42 11.760000109672546 +8.45 8.449999809265137 +8.45 8.449999809265137 +8.45 10.319999814033508 +8.45 16.009999752044678 +8.57 8.569999694824219 +8.61 8.609999656677246 +8.67 8.670000076293945 +8.71 8.710000038146973 +8.79 8.789999961853027 +8.91 13.480000019073486 +9.04 9.039999961853027 +9.13 9.130000114440918 +9.19 15.479999542236328 +9.22 9.220000267028809 +9.25 9.25 +9.26 9.260000228881836 +9.35 12.350000381469727 +9.48 9.479999542236328 +9.56 12.480000495910645 +9.57 9.569999694824219 +9.57 9.569999694824219 +9.68 9.680000305175781 +9.7 9.699999809265137 +9.71 17.25 +9.74 9.739999771118164 +9.8 9.800000190734863 +9.81 9.8100004196167 +9.93 9.930000305175781 +10.09 10.09000015258789 +10.09 10.09000015258789 +10.13 15.640000343322754 +10.16 18.139999866485596 +10.17 14.970000267028809 +10.19 10.1899995803833 +10.2 10.199999809265137 +10.22 10.220000267028809 +10.25 18.859999656677246 +10.26 10.260000228881836 +10.29 10.289999961853027 +10.6 10.600000381469727 +10.66 10.65999984741211 +10.67 10.670000076293945 +10.73 10.729999542236328 +11.15 20.27999973297119 +11.18 13.360000371932983 +11.19 18.979999542236328 +11.22 11.220000267028809 +11.34 16.62000036239624 +11.55 11.550000190734863 +11.57 11.569999694824219 +11.68 17.080000400543213 +11.82 11.819999694824219 +11.89 11.890000343322754 +11.91 11.90999984741211 +12.02 12.020000457763672 +12.16 12.15999984741211 +12.19 20.149999618530273 +12.32 12.319999694824219 +12.42 16.27999997138977 +12.44 12.4399995803833 +12.45 16.799999713897705 +12.46 12.460000038146973 +12.5 15.460000038146973 +12.54 12.539999961853027 +12.85 12.850000381469727 +12.9 12.899999618530273 +13.01 28.47000026702881 +13.1 23.030000686645508 +13.15 20.96999979019165 +13.35 13.350000381469727 +13.87 13.869999885559082 +13.89 13.890000343322754 +13.94 13.9399995803833 +13.99 13.989999771118164 +14.13 24.22000026702881 +14.21 14.210000038146973 +14.3 29.270000457763672 +14.44 14.4399995803833 +14.84 14.84000015258789 +14.92 14.920000076293945 +14.92 25.18000030517578 +14.93 30.40999984741211 +15.1 17.700000286102295 +15.15 24.40999984741211 +15.18 26.730000495910645 +15.22 15.220000267028809 +15.26 15.260000228881836 +15.3 25.5 +15.37 15.369999885559082 +15.45 15.449999809265137 +15.63 28.110000610351562 +15.75 15.75 +15.81 15.8100004196167 +15.86 25.079999923706055 +15.9 21.34999942779541 +15.92 15.920000076293945 +16.08 22.75 +16.09 16.520000159740448 +16.24 27.809999465942383 +16.25 19.419999957084656 +16.48 16.479999542236328 +16.69 16.690000534057617 +16.99 16.989999771118164 +16.99 42.489999771118164 +17.16 21.12999987602234 +17.37 31.360000610351562 +17.74 19.049999713897705 +17.79 45.60000038146973 +17.87 18.710000813007355 +18.2 18.200000762939453 +18.5 31.350000381469727 +18.56 18.559999465942383 +18.63 26.589999198913574 +18.63 30.389999270439148 +18.86 18.96000061184168 +18.89 18.889999389648438 +18.93 18.93000030517578 +19.0 35.62000036239624 +19.03 19.030000686645508 +19.06 19.059999465942383 +19.06 19.059999465942383 +19.13 45.719998359680176 +19.14 19.139999389648438 +19.28 27.600000381469727 +19.69 36.49000024795532 +20.07 46.80000019073486 +20.38 51.72999954223633 +20.64 20.639999389648438 +20.67 31.830000400543213 +20.79 20.790000915527344 +20.81 20.809999465942383 +20.82 21.419999718666077 +20.82 26.359999656677246 +21.18 21.18000030517578 +21.19 21.190000534057617 +21.23 21.229999542236328 +21.28 29.350000381469727 +21.32 21.31999969482422 +21.45 40.510000228881836 +21.49 30.739999771118164 +21.61 37.0600004196167 +21.7 27.58000087738037 +21.8 21.799999237060547 +21.94 23.940000593662262 +22.01 28.850000381469727 +22.08 22.079999923706055 +22.12 22.1200008392334 +22.12 22.1200008392334 +22.25 22.25 +22.27 22.270000457763672 +22.36 22.360000610351562 +22.68 22.68000030517578 +22.78 47.19000053405762 +22.85 33.070000648498535 +22.85 43.980000257492065 +22.94 35.38000011444092 +23.07 23.06999969482422 +23.13 28.799999237060547 +23.17 44.58999979496002 +23.19 23.190000534057617 +23.44 23.440000534057617 +23.45 24.74000072479248 +23.6 33.16000032424927 +23.77 23.770000457763672 +23.96 23.959999084472656 +24.02 24.020000457763672 +24.28 43.310001373291016 +24.49 42.62999963760376 +24.52 32.59000015258789 +24.73 45.369998931884766 +24.79 24.790000915527344 +24.8 34.369998931884766 +24.83 36.05000019073486 +24.86 65.3700008392334 +25.11 44.170000076293945 +25.28 25.280000686645508 +25.37 48.05000114440918 +25.42 40.78999996185303 +25.55 26.62999927997589 +25.67 37.69000053405762 +25.88 61.49999952316284 +26.08 26.079999923706055 +26.39 34.959999084472656 +26.43 26.43000030517578 +26.47 31.389999389648438 +26.49 26.489999771118164 +26.49 48.56999969482422 +26.64 64.32999992370605 +26.71 36.999999046325684 +26.73 45.69000015407801 +26.76 26.760000228881836 +27.07 28.649999737739563 +27.12 32.20000076293945 +27.3 70.61000061035156 +27.31 56.579999923706055 +27.63 27.6299991607666 +27.66 27.65999984741211 +27.72 46.60999870300293 +27.87 27.8700008392334 +28.11 44.59000015258789 +28.31 52.079999923706055 +28.45 74.05000114440918 +28.5 35.36999988555908 +28.56 40.71999931335449 +28.69 28.690000534057617 +28.71 55.46999931335449 +28.79 28.790000915527344 +28.89 56.489999771118164 +28.95 33.410000801086426 +29.02 56.64999961853027 +29.24 99.85000038146973 +29.36 62.52000093460083 +29.4 72.02999925613403 +29.41 64.77999973297119 +29.54 29.540000915527344 +29.59 37.37000012397766 +29.78 66.77999973297119 +30.25 30.32999999821186 +30.36 30.360000610351562 +30.37 31.660000830888748 +30.61 30.610000610351562 +30.62 102.65000009536743 +30.63 30.6299991607666 +30.65 60.19000053405762 +30.71 49.849998474121094 +30.81 55.989999771118164 +31.01 31.010000228881836 +31.15 31.149999618530273 +31.4 31.399999618530273 +31.61 31.610000610351562 +31.67 40.46000003814697 +31.77 42.09000027179718 +31.86 31.860000610351562 +31.91 78.51999855041504 +32.01 60.47999858856201 +32.18 58.61000061035156 +32.2 53.55000019073486 +32.23 42.89999961853027 +32.25 59.83000087738037 +32.37 62.99999809265137 +32.41 32.40999984741211 +32.47 41.14000129699707 +32.52 95.0400013923645 +32.75 56.19000053405762 +32.89 80.07999992370605 +32.92 47.7599983215332 +33.36 45.27000045776367 +33.52 60.010000228881836 +33.55 63.939998507499695 +33.58 54.55000162124634 +33.67 33.66999816894531 +33.76 112.27999687194824 +33.83 59.110002517700195 +33.85 47.719998359680176 +33.87 37.48999881744385 +34.03 71.51999759674072 +34.21 71.57999920845032 +34.35 34.349998474121094 +34.41 59.20000076293945 +34.58 34.58000183105469 +34.73 34.72999954223633 +34.97 45.160000801086426 +35.0 35.0 +35.08 36.060001850128174 +35.13 39.600000858306885 +35.17 64.01999855041504 +35.17 66.52999877929688 +35.56 37.63000130653381 +35.62 80.20999872684479 +35.65 56.83000183105469 +35.68 52.20000046491623 +35.72 98.71999931335449 +35.8 79.96999931335449 +35.89 81.2599983215332 +36.22 55.20000076293945 +36.26 36.2599983215332 +36.58 64.45000267028809 +36.7 73.76000118255615 +36.79 36.790000915527344 +36.89 71.8499984741211 +36.95 36.95000076293945 +37.07 37.06999969482422 +37.1 51.039998054504395 +37.14 53.41999936103821 +37.14 61.15999984741211 +37.24 47.04000186920166 +37.59 50.94000053405762 +37.6 84.39999866485596 +37.72 57.14000117778778 +37.78 68.10999877750874 +37.8 57.94999885559082 +37.85 50.38999843597412 +37.9 77.50000238418579 +38.05 40.24999928474426 +38.05 47.859999656677246 +38.3 98.48999977111816 +38.33 112.09000301361084 +38.57 55.81999969482422 +38.62 73.99999904632568 +38.79 95.44000053405762 +38.85 97.45999908447266 +38.88 48.58000087738037 +38.94 71.3499984741211 +39.01 39.0099983215332 +39.03 57.73999959230423 +39.18 99.6599988937378 +39.21 71.79999923706055 +39.34 39.34000015258789 +39.69 55.69999837875366 +39.81 74.81000137329102 +39.82 39.81999969482422 +39.83 119.80000114440918 +39.87 62.22999954223633 +39.9 64.98000144958496 +39.98 39.97999954223633 +40.0 84.59000015258789 +40.04 101.20000076293945 +40.17 80.41999745368958 +40.24 67.9000015258789 +40.42 78.04999947547913 +40.44 117.94000101089478 +40.78 49.8199987411499 +40.8 40.79999923706055 +40.98 92.01999759674072 +41.2 58.280001163482666 +41.29 41.290000915527344 +41.29 112.87000012397766 +41.31 53.08000135421753 +41.34 53.230000495910645 +41.34 115.33999919891357 +41.36 41.36000061035156 +41.44 85.41999888420105 +41.45 91.29999923706055 +41.62 41.619998931884766 +41.68 109.58000183105469 +41.71 83.06999969482422 +41.81 89.8600025177002 +41.85 76.21999740600586 +41.87 47.48999881744385 +41.89 41.88999938964844 +42.24 72.6500015258789 +42.31 52.91000175476074 +42.42 154.69999504089355 +42.48 71.12999927997589 +42.51 61.069997787475586 +42.55 87.71000003814697 +42.56 71.91000175476074 +42.67 71.45999908447266 +42.76 42.7599983215332 +42.85 85.33999824523926 +43.01 109.53999710083008 +43.02 46.84000039100647 +43.13 43.130001068115234 +43.16 58.079999923706055 +43.17 43.16999816894531 +43.19 156.05999875068665 +43.31 64.10000228881836 +43.37 56.84999895095825 +43.57 43.71999970078468 +43.71 108.69000053405762 +43.73 108.50999927520752 +43.92 48.319998145103455 +44.1 67.1299991607666 +44.22 103.33000373840332 +44.27 74.88000106811523 +44.43 106.65999984741211 +44.57 59.00999927520752 +45.06 45.060001373291016 +45.1 45.099998474121094 +45.19 45.189998626708984 +45.19 117.10000038146973 +45.24 109.34000396728516 +45.34 129.73999881744385 +45.35 109.28999698162079 +45.42 77.61999893188477 +45.45 45.45000076293945 +45.56 137.57999897003174 +45.59 82.54000091552734 +45.68 55.25 +45.92 90.97999954223633 +45.99 47.07000172138214 +46.02 82.27999877929688 +46.09 46.09000015258789 +46.1 98.17999839782715 +46.15 72.7800008058548 +46.18 74.29000091552734 +46.21 52.75999927520752 +46.27 85.61000061035156 +46.43 106.26000118255615 +46.45 110.90000343322754 +46.62 78.0099983215332 +46.8 80.46999740600586 +46.86 62.08000087738037 +46.87 70.80999952554703 +46.88 106.08000183105469 +46.97 88.1100025177002 +47.08 148.28000259399414 +47.27 50.60000038146973 +47.32 118.12999922037125 +47.57 90.32999801635742 +47.59 104.17000007629395 +47.69 99.88999909162521 +47.88 47.880001068115234 +48.01 91.72999802231789 +48.08 79.69000244140625 +48.11 48.11000061035156 +48.15 65.8500018119812 +48.22 105.07000017166138 +48.23 139.52999877929688 +48.25 48.25 +48.28 98.669997215271 +48.37 185.9499979019165 +48.45 48.45000076293945 +48.45 94.54000091552734 +48.52 146.69999885559082 +48.59 89.30999946594238 +49.12 49.119998931884766 +49.28 123.56999969482422 +49.44 110.93999814987183 +49.68 73.63999938964844 +49.77 50.33000046014786 +49.78 66.46999931335449 +50.02 63.380000829696655 +50.08 156.16000366210938 +50.09 106.28000068664551 +50.26 72.05999755859375 +50.28 50.279998779296875 +50.31 117.44000053405762 +50.32 90.13999938964844 +50.4 96.11999988555908 +50.41 98.72999799251556 +50.66 55.37999963760376 +50.7 131.11999821662903 +50.83 98.69000148773193 +50.92 53.70999813079834 +50.96 103.71999835968018 +51.25 67.0 +51.29 87.35000276565552 +51.29 124.93000030517578 +51.72 97.17000198364258 +51.79 139.90000343322754 +51.84 168.94000053405762 +51.85 171.64999961853027 +52.17 206.86999320983887 +52.23 177.1599998474121 +52.44 88.48999881744385 +52.5 105.41000175476074 +52.53 64.98999881744385 +52.72 52.720001220703125 +52.73 74.04999923706055 +52.85 89.63999938964844 +52.87 130.87999725341797 +53.02 100.50999927520752 +53.06 259.9299945831299 +53.18 53.18000030517578 +53.27 53.27000045776367 +53.59 53.59000015258789 +53.78 139.38999938964844 +53.93 57.890000343322754 +53.94 63.529998779296875 +54.1 152.7699956893921 +54.31 77.38000106811523 +54.34 125.46999943256378 +54.43 132.04999923706055 +54.44 103.01999950408936 +54.47 186.52000045776367 +54.73 63.179999351501465 +54.75 112.82999992370605 +54.83 110.82000160217285 +54.99 160.40000343322754 +55.1 161.35999965667725 +55.18 215.58000373840332 +55.2 126.65999984741211 +55.39 137.6699981689453 +55.51 74.55999803543091 +55.63 96.43000030517578 +55.99 187.10999989509583 +56.04 150.5800018310547 +56.07 118.15000057220459 +56.1 135.79000091552734 +56.15 144.64000034332275 +56.33 61.77000188827515 +56.62 78.88999938964844 +56.68 154.13999938964844 +56.81 169.64000129699707 +57.08 69.98000144958496 +57.11 168.0100040435791 +57.12 100.28999710083008 +57.23 65.9399995803833 +57.25 133.46999740600586 +57.29 112.54000091552734 +57.35 110.89999866485596 +57.37 115.109998524189 +57.46 147.78999710083008 +57.64 112.19000101089478 +57.67 57.66999816894531 +57.89 111.15999984741211 +57.93 68.02000045776367 +58.0 123.9399995803833 +58.08 58.08000183105469 +58.09 206.37000274658203 +58.13 105.84999942779541 +58.43 165.0900001525879 +58.52 167.0299997329712 +58.66 136.04000091552734 +58.67 205.36999702453613 +58.75 90.41000083088875 +58.86 165.14000129699707 +59.07 87.86999893188477 +59.16 224.25 +59.21 90.35999870300293 +59.34 127.44999893009663 +59.43 106.50000202655792 +59.45 67.90000057220459 +59.45 197.11999893188477 +59.5 149.63999938964844 +59.55 61.459999203681946 +59.61 85.97000026702881 +59.62 113.3299970626831 +59.68 73.89000034332275 +59.68 94.40999984741211 +59.7 193.1699981689453 +59.71 60.22999906539917 +59.83 145.17000007629395 +59.87 228.80999946594238 +59.99 134.04000282287598 +60.02 66.76000022888184 +60.06 60.060001373291016 +60.12 113.34999942779541 +60.13 214.27000045776367 +60.22 108.10000228881836 +60.26 105.94999847561121 +60.26 165.32999849319458 +60.53 66.37999868392944 +60.6 82.8499984741211 +60.71 72.04999899864197 +60.85 132.36999607086182 +61.21 160.86999797821045 +61.7 127.55000257492065 +61.86 248.9700005054474 +61.88 112.15999984741211 +61.92 125.29999899864197 +61.94 119.6099967956543 +62.14 110.59000015258789 +62.2 149.91000080108643 +62.23 111.3499984741211 +62.3 158.41999912261963 +62.39 110.95999908447266 +62.52 123.97999966144562 +62.72 123.78999900817871 +62.74 153.10000038146973 +62.85 167.01999855041504 +62.9 256.0699996948242 +62.92 129.3899974822998 +63.12 93.47999954223633 +63.33 135.38999938964844 +63.35 116.93999862670898 +63.42 172.76000213623047 +63.51 123.51999855041504 +63.9 135.70000076293945 +64.0 191.55000257492065 +64.22 86.97000122070312 +64.25 131.25 +64.3 122.3800048828125 +64.36 85.59000015258789 +64.46 134.44000053405762 +64.65 143.54000091552734 +64.67 121.15999794006348 +64.77 214.40999603271484 +64.87 194.61000156402588 +64.95 324.87999153137207 +65.02 175.6099967956543 +65.02 259.6299982070923 +65.38 168.7100009918213 +65.43 112.27000069618225 +65.43 289.6800003051758 +65.44 192.89000137150288 +65.55 66.16000306606293 +65.62 139.67000198364258 +65.7 65.69999694824219 +65.72 77.54000091552734 +66.17 177.10999631881714 +66.17 200.60999870300293 +66.36 131.73000144958496 +66.51 83.50000190734863 +66.61 66.61000061035156 +66.61 78.93000030517578 +66.67 129.84999752044678 +66.89 99.96000003814697 +67.12 67.12000274658203 +67.18 234.21000003814697 +67.26 77.9900016784668 +67.38 178.72999572753906 +67.45 197.29999446868896 +67.48 268.0900020599365 +67.59 272.95999336242676 +67.94 125.89000129699707 +67.98 123.36000299453735 +68.01 124.84000396728516 +68.04 166.76000022888184 +68.22 181.57000064849854 +68.25 113.52000045776367 +68.25 163.2900013923645 +68.32 247.04999542236328 +68.41 157.72000312805176 +68.5 156.36999893188477 +68.81 93.02999782562256 +68.85 160.57999649643898 +68.89 89.69999885559082 +68.95 140.99999594688416 +68.96 192.4799976348877 +69.32 156.29000091552734 +69.53 239.17000007629395 +69.74 246.84999418258667 +69.8 101.63000345230103 +69.88 220.45999908447266 +69.96 83.84999942779541 +69.97 169.86000031232834 +70.0 236.76000022888184 +70.04 196.70000076293945 +70.06 92.17999839782715 +70.24 213.77999877929688 +70.35 247.5099983215332 +70.38 179.95999908447266 +70.39 164.79999923706055 +70.52 181.33999824523926 +70.53 70.52999877929688 +70.56 110.53999710083008 +70.85 223.94999885559082 +70.89 149.81999969482422 +70.93 160.79000282287598 +71.01 92.20000267028809 +71.07 112.36000061035156 +71.13 135.149995803833 +71.19 136.1800012588501 +71.26 318.7700004577637 +71.31 231.88999405503273 +71.32 145.21000003814697 +71.35 145.909996509552 +71.5 217.409996509552 +71.54 71.54000091552734 +71.55 90.41000270843506 +71.68 72.9500002861023 +71.68 227.97000122070312 +71.78 189.71999979019165 +71.8 103.66000366210938 +71.89 180.57999992370605 +72.04 156.63000106811523 +72.18 181.46999728679657 +72.51 264.0600047111511 +72.53 265.42000015079975 +72.56 132.61999893188477 +72.62 205.2400016784668 +72.79 72.79000091552734 +72.98 286.76000213623047 +73.18 123.78000068664551 +73.32 175.96999979019165 +73.48 144.8300018310547 +73.63 320.47999143600464 +73.65 114.11000156402588 +73.68 161.0300030708313 +73.88 291.28999376296997 +73.93 156.77999877929688 +74.0 179.8499994277954 +74.02 89.27999687194824 +74.15 74.1500015258789 +74.19 74.47000244259834 +74.19 122.44000244140625 +74.3 159.72000193595886 +74.42 219.62999820709229 +74.45 203.839994430542 +74.52 271.6399955749512 +74.53 342.6200008392334 +74.59 157.12999725341797 +74.62 163.89999961853027 +74.72 139.05000114440918 +74.78 230.83999752998352 +75.03 174.98999881744385 +75.1 214.62999725341797 +75.19 149.6600048840046 +75.29 93.4900016784668 +75.35 256.91999912261963 +75.42 153.46999764442444 +75.66 225.4800033569336 +75.73 236.60000133514404 +75.83 199.4000015258789 +75.88 243.89000129699707 +76.05 179.770001411438 +76.1 257.43999671936035 +76.28 168.48000144958496 +76.28 177.9100022315979 +76.33 363.09000396728516 +76.52 207.39999389648438 +76.69 212.87000370025635 +76.7 307.5399944782257 +76.71 132.17999839782715 +76.72 166.42000007629395 +76.72 258.1899985074997 +76.74 102.81999778747559 +76.92 401.7999897003174 +76.93 384.4699947834015 +77.02 296.6499948501587 +77.1 187.63999557495117 +77.36 281.19999504089355 +77.42 217.32000160217285 +77.57 189.66000270843506 +77.66 267.380003452301 +77.81 178.31999683380127 +77.84 350.7999897003174 +77.89 249.5399990081787 +77.97 178.2599983215332 +78.21 189.1099977493286 +78.26 284.6300048828125 +78.28 306.25 +78.3 168.71000388264656 +78.31 161.80999946594238 +78.62 275.3200035095215 +78.64 94.55999946594238 +78.73 310.6199974119663 +78.89 175.31999969482422 +78.98 293.25000381469727 +79.12 276.419997215271 +79.19 194.30000096559525 +79.21 237.62999820709229 +79.38 151.22999572753906 +79.42 124.6099967956543 +79.48 116.55000305175781 +79.48 200.64000129699707 +79.49 294.1199951171875 +79.54 145.23999786376953 +79.55 254.54000186920166 +79.75 215.13999938964844 +79.83 294.9700012207031 +79.96 79.95999908447266 +79.97 266.4900016784668 +79.99 219.6599998474121 +80.23 248.71000480651855 +80.3 133.4800033569336 +80.3 305.7800064086914 +80.46 249.17000296711922 +80.52 136.21999502182007 +80.58 261.16000175476074 +80.6 281.23999977111816 +80.71 275.0100000500679 +80.74 240.4599997997284 +80.84 142.60999822616577 +80.92 340.5499963760376 +80.96 372.2499928474426 +80.97 93.32000160217285 +80.99 317.74999809265137 +81.17 220.55999755859375 +81.32 158.86000061035156 +81.32 206.16000366210938 +81.47 198.91000175476074 +81.58 173.76000022888184 +81.64 207.5300006866455 +81.66 261.5100030899048 +82.24 243.59999752044678 +82.3 157.18000411987305 +82.34 214.0699977874756 +82.41 135.83000302314758 +82.52 240.23999977111816 +82.55 139.6900042295456 +82.56 175.8799991607666 +82.72 232.63000202178955 +82.97 101.9000015258789 +83.08 181.57000160217285 +83.27 83.2699966430664 +83.33 348.75000198185444 +83.4 196.729998588562 +83.54 299.12000465393066 +83.57 139.38999938964844 +83.58 163.54000091552734 +83.87 263.64000415802 +83.92 432.67000015079975 +83.93 358.9400003552437 +84.03 240.19000244140625 +84.23 209.53000235557556 +84.31 157.0999984741211 +84.38 220.1699981689453 +84.4 443.3400018811226 +84.69 249.4900016784668 +84.72 105.00000095367432 +84.83 157.4800033569336 +85.0 221.21999502182007 +85.03 283.9400005340576 +85.1 217.27999687194824 +85.14 266.7100009918213 +85.23 277.7100009918213 +85.49 261.4599976539612 +85.49 325.9499976634979 +85.51 165.59000205993652 +85.74 341.80999755859375 +85.76 160.57000350952148 +85.87 357.5099983215332 +85.9 202.45000457763672 +86.0 322.60000133514404 +86.22 152.8300018310547 +86.23 152.99000358581543 +86.63 381.5999984741211 +86.69 138.42000198364258 +86.92 147.14999723434448 +86.93 97.59000015258789 +86.93 218.18000030517578 +87.14 219.50999546051025 +87.22 211.01000022888184 +87.4 137.73000198602676 +87.48 153.86000204086304 +87.57 307.0799951553345 +87.61 321.82000064849854 +87.67 248.4600009918213 +87.83 130.96000289916992 +87.94 134.9800043106079 +87.99 227.0399990081787 +88.02 406.7899971008301 +88.05 97.53000259399414 +88.07 210.45000457763672 +88.17 245.6500015258789 +88.22 369.4600009918213 +88.36 305.6399974822998 +88.47 213.9400006532669 +88.48 211.84000635147095 +88.55 199.71000289916992 +88.77 187.49999463558197 +88.78 337.23999977111816 +88.8 254.1300015449524 +88.91 276.40999829769135 +89.01 98.69000244140625 +89.03 431.6499996185303 +89.1 113.83999919891357 +89.15 234.32000160217285 +89.28 343.82000064849854 +89.38 496.16999435424805 +89.53 153.05999755859375 +89.55 89.55000305175781 +89.55 96.27000284194946 +89.55 172.62000274658203 +89.8 205.1400022506714 +89.81 225.64000058174133 +89.93 89.93000030517578 +90.05 90.05000305175781 +90.05 247.1500015258789 +90.07 137.82999801635742 +90.12 234.76000308990479 +90.2 208.32999616861343 +90.25 386.8999948501587 +90.28 334.17000007629395 +90.35 158.36999893188477 +90.38 148.65999841690063 +90.51 268.7700004577637 +90.56 298.8899937272072 +90.69 181.1000051498413 +90.69 523.360002592206 +90.73 260.59000366926193 +90.77 203.03999733924866 +91.05 263.67000579833984 +91.16 298.55999755859375 +91.42 196.41999912261963 +91.48 144.56000471115112 +91.53 301.9800033569336 +91.61 215.59000027179718 +91.63 432.17999362945557 +91.78 330.9499988555908 +91.88 277.8299951553345 +91.97 205.4900016784668 +92.05 317.69000363349915 +92.11 307.70000088214874 +92.33 92.33000183105469 +92.37 226.410005569458 +92.4 265.1600036621094 +92.55 295.59000039100647 +92.61 249.38999938964844 +92.82 182.87000274658203 +92.96 185.98999691009521 +92.98 494.779993057251 +93.03 165.80999958515167 +93.09 456.1800003051758 +93.11 427.2800006866455 +93.61 465.8599934577942 +93.64 314.1999969482422 +93.73 475.3300018310547 +94.08 187.56000137329102 +94.15 362.9200019836426 +94.25 308.1900006532669 +94.27 351.70999336242676 +94.31 216.75 +94.33 193.02000427246094 +94.34 320.75000190734863 +94.38 260.18999683856964 +94.43 174.89999771118164 +94.54 340.19000244140625 +94.66 192.19000625610352 +94.68 189.23999977111816 +94.68 344.1700019836426 +94.72 274.6800003051758 +95.07 417.67000102996826 +95.11 393.99999433755875 +95.28 140.37999725341797 +95.33 244.99000671505928 +95.34 192.92999649047852 +95.38 230.36000156402588 +95.48 406.1000007688999 +95.53 248.58999633789062 +95.53 252.71000289916992 +95.81 134.81999588012695 +95.81 551.9899978637695 +95.84 274.1599931716919 +96.09 191.52999687194824 +96.23 398.2100067138672 +96.25 261.8400020599365 +96.29 199.10999870300293 +96.38 323.4199962615967 +96.62 309.4900064468384 +96.68 336.87000274658203 +96.73 272.0500030517578 +96.78 190.27000045776367 +96.91 180.18000030517578 +96.94 216.54999923706055 +97.09 428.03999519348145 +97.24 346.41000083088875 +97.26 373.67999935150146 +97.39 257.9600028991699 +97.46 449.1699924468994 +97.51 132.09000396728516 +97.56 97.55999755859375 +97.57 592.3499927520752 +97.65 196.34000301361084 +97.68 258.7100033760071 +97.71 175.70000076293945 +97.81 297.20999908447266 +97.83 396.38999938964844 +97.87 254.50000381469727 +98.18 105.29000043869019 +98.22 438.4100036621094 +98.23 525.5100040435791 +98.31 335.93999576568604 +98.48 286.0400047302246 +98.51 132.86000061035156 +98.57 404.3500061035156 +98.72 263.8600025177002 +98.96 288.1999988555908 +99.13 243.6900019645691 +99.15 210.11000061035156 +99.21 347.7999954223633 +99.24 537.6500015258789 +99.29 291.48000717163086 +99.36 106.41000080108643 +99.62 417.3700008392334 +99.65 185.62000179290771 +99.67 180.92999649047852 +99.68 230.6400032043457 +99.91 367.2900071144104 +99.92 376.32999646663666 diff --git a/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-7-20fdc99aa046b2c41d9b85ab338c749c b/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-7-20fdc99aa046b2c41d9b85ab338c749c new file mode 100644 index 0000000000000..1a4528978b74f --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-7-20fdc99aa046b2c41d9b85ab338c749c @@ -0,0 +1,1049 @@ + 65560 3.1 + 65718 2.38 + 65740 3.04 +alice allen 65662 1.55 +alice allen 65720 1.79 +alice allen 65758 1.98 +alice brown 65696 0.13 +alice carson 65559 4.2 +alice davidson 65547 1.51 +alice falkner 65669 4.19 +alice garcia 65613 0.72 +alice hernandez 65737 0.92 +alice hernandez 65784 2.09 +alice johnson 65739 2.55 +alice king 65660 3.84 +alice king 65734 2.96 +alice king 65738 2.14 +alice laertes 65669 0.28 +alice laertes 65671 1.16 +alice miller 65590 4.95 +alice nixon 65586 2.98 +alice nixon 65595 2.36 +alice nixon 65604 2.45 +alice ovid 65737 3.2 +alice polk 65548 1.23 +alice quirinius 65636 0.23 +alice quirinius 65728 0.82 +alice robinson 65606 3.99 +alice robinson 65789 4.35 +alice steinbeck 65578 4.72 +alice steinbeck 65673 3.97 +alice steinbeck 65786 3.92 +alice underhill 65750 2.06 +alice van buren 65562 2.43 +alice xylophone 65578 2.22 +alice xylophone 65585 2.11 +alice xylophone 65599 2.92 +alice zipper 65553 3.78 +alice zipper 65662 2.61 +alice zipper 65766 3.12 +bob brown 65584 2.09 +bob brown 65777 1.62 +bob brown 65783 2.4 +bob carson 65713 3.87 +bob davidson 65664 4.25 +bob davidson 65693 3.54 +bob davidson 65768 2.91 +bob ellison 65591 2.23 +bob ellison 65624 1.69 +bob ellison 65721 1.69 +bob ellison 65760 1.69 +bob falkner 65789 0.21 +bob garcia 65585 3.91 +bob garcia 65598 2.64 +bob garcia 65673 3.26 +bob garcia 65754 3.29 +bob garcia 65782 2.86 +bob hernandez 65557 3.72 +bob ichabod 65549 1.39 +bob king 65715 2.76 +bob king 65757 1.71 +bob king 65783 2.24 +bob laertes 65602 4.98 +bob laertes 65663 3.56 +bob miller 65608 4.95 +bob ovid 65564 1.23 +bob ovid 65619 1.53 +bob ovid 65686 1.84 +bob ovid 65726 2.38 +bob polk 65594 0.28 +bob quirinius 65700 3.82 +bob steinbeck 65637 0.22 +bob van buren 65778 2.89 +bob white 65543 4.75 +bob white 65605 2.89 +bob xylophone 65574 1.7 +bob xylophone 65666 2.51 +bob young 65556 0.95 +bob zipper 65559 3.18 +bob zipper 65633 3.2 +bob zipper 65739 3.24 +calvin allen 65669 2.3 +calvin brown 65537 1.1 +calvin brown 65580 2.82 +calvin brown 65677 2.57 +calvin carson 65637 1.33 +calvin davidson 65541 1.98 +calvin davidson 65564 1.6 +calvin ellison 65667 1.85 +calvin falkner 65573 3.52 +calvin falkner 65596 3.31 +calvin falkner 65738 2.36 +calvin falkner 65762 2.26 +calvin falkner 65778 2.7 +calvin falkner 65784 2.98 +calvin garcia 65664 2.9 +calvin hernandez 65578 2.08 +calvin johnson 65731 4.69 +calvin laertes 65570 4.7 +calvin laertes 65684 3.68 +calvin nixon 65654 2.74 +calvin nixon 65724 3.27 +calvin nixon 65749 2.88 +calvin ovid 65554 3.34 +calvin ovid 65643 2.38 +calvin ovid 65663 2.74 +calvin ovid 65715 2.47 +calvin polk 65731 4.36 +calvin quirinius 65741 4.0 +calvin quirinius 65769 2.2 +calvin robinson 65581 3.32 +calvin steinbeck 65680 1.29 +calvin steinbeck 65762 1.3 +calvin steinbeck 65779 1.98 +calvin thompson 65560 4.08 +calvin thompson 65640 3.33 +calvin underhill 65732 2.41 +calvin van buren 65552 1.05 +calvin van buren 65771 1.33 +calvin white 65553 4.7 +calvin white 65561 2.68 +calvin xylophone 65575 4.6 +calvin xylophone 65596 4.77 +calvin xylophone 65713 3.23 +calvin young 65574 0.27 +calvin young 65746 0.9 +calvin zipper 65669 4.4 +calvin zipper 65739 2.29 +david allen 65588 3.86 +david allen 65617 3.18 +david brown 65637 1.17 +david brown 65760 1.01 +david davidson 65559 1.37 +david davidson 65756 1.57 +david davidson 65778 1.89 +david davidson 65779 2.43 +david ellison 65634 3.23 +david ellison 65724 2.95 +david ellison 65724 2.95 +david hernandez 65763 1.15 +david ichabod 65699 1.67 +david ichabod 65715 1.26 +david laertes 65762 1.38 +david nixon 65536 1.27 +david ovid 65623 0.23 +david ovid 65628 1.15 +david quirinius 65697 1.14 +david quirinius 65759 1.65 +david quirinius 65779 1.93 +david robinson 65762 3.51 +david robinson 65775 3.38 +david thompson 65550 3.3 +david underhill 65602 0.12 +david underhill 65662 2.29 +david underhill 65751 2.43 +david van buren 65625 1.55 +david van buren 65634 3.25 +david white 65678 0.17 +david xylophone 65537 1.07 +david xylophone 65670 0.72 +david xylophone 65764 0.94 +david young 65551 4.51 +david young 65694 2.74 +ethan allen 65747 3.61 +ethan brown 65539 2.9 +ethan brown 65617 1.59 +ethan brown 65685 2.17 +ethan brown 65685 2.17 +ethan brown 65722 2.64 +ethan brown 65733 2.75 +ethan carson 65742 2.84 +ethan ellison 65714 4.87 +ethan ellison 65732 3.9 +ethan falkner 65577 3.61 +ethan falkner 65614 1.95 +ethan garcia 65736 4.63 +ethan hernandez 65618 0.46 +ethan johnson 65536 1.76 +ethan king 65614 0.92 +ethan laertes 65562 2.89 +ethan laertes 65597 3.45 +ethan laertes 65628 3.15 +ethan laertes 65643 3.53 +ethan laertes 65680 3.18 +ethan laertes 65745 3.36 +ethan laertes 65760 3.01 +ethan miller 65712 1.97 +ethan nixon 65766 4.1 +ethan ovid 65697 3.81 +ethan polk 65589 0.7 +ethan polk 65615 1.92 +ethan polk 65622 2.0 +ethan polk 65622 2.0 +ethan quirinius 65542 4.64 +ethan quirinius 65591 3.97 +ethan quirinius 65706 2.88 +ethan robinson 65547 2.2 +ethan robinson 65659 2.17 +ethan underhill 65570 2.45 +ethan van buren 65572 1.11 +ethan white 65677 3.42 +ethan white 65788 4.09 +ethan xylophone 65595 4.66 +ethan zipper 65593 2.1 +ethan zipper 65680 2.53 +fred davidson 65552 0.37 +fred davidson 65595 2.31 +fred davidson 65721 2.65 +fred ellison 65548 4.32 +fred ellison 65691 3.44 +fred ellison 65771 3.1 +fred falkner 65637 4.59 +fred falkner 65648 2.79 +fred falkner 65783 2.35 +fred hernandez 65541 3.87 +fred ichabod 65572 1.45 +fred ichabod 65789 1.68 +fred johnson 65758 3.86 +fred king 65694 4.05 +fred king 65745 4.5 +fred laertes 65769 3.89 +fred miller 65536 2.59 +fred nixon 65560 2.52 +fred nixon 65612 2.0 +fred nixon 65703 1.35 +fred nixon 65705 1.16 +fred polk 65603 2.9 +fred polk 65656 1.65 +fred polk 65701 1.75 +fred polk 65706 1.4 +fred quirinius 65697 1.91 +fred quirinius 65782 3.24 +fred robinson 65623 2.6 +fred steinbeck 65544 2.0 +fred steinbeck 65651 3.32 +fred steinbeck 65755 3.51 +fred underhill 65629 2.56 +fred van buren 65537 4.97 +fred van buren 65561 3.28 +fred van buren 65745 3.24 +fred van buren 65789 3.27 +fred white 65589 3.16 +fred young 65594 0.34 +fred young 65773 1.56 +fred zipper 65553 0.37 +gabriella allen 65646 1.68 +gabriella allen 65677 1.6 +gabriella brown 65704 0.02 +gabriella brown 65753 1.86 +gabriella carson 65586 0.37 +gabriella davidson 65565 3.45 +gabriella ellison 65706 1.15 +gabriella ellison 65716 2.06 +gabriella falkner 65623 2.09 +gabriella falkner 65711 2.48 +gabriella falkner 65767 1.82 +gabriella garcia 65571 3.17 +gabriella hernandez 65587 0.74 +gabriella hernandez 65717 0.96 +gabriella ichabod 65559 0.63 +gabriella ichabod 65633 2.42 +gabriella ichabod 65702 3.27 +gabriella ichabod 65712 3.6 +gabriella ichabod 65717 3.56 +gabriella king 65651 3.59 +gabriella king 65657 2.17 +gabriella laertes 65781 2.81 +gabriella miller 65646 3.47 +gabriella ovid 65556 1.23 +gabriella ovid 65583 1.95 +gabriella polk 65701 3.58 +gabriella polk 65790 2.08 +gabriella steinbeck 65582 3.6 +gabriella steinbeck 65653 2.7 +gabriella thompson 65682 1.78 +gabriella thompson 65755 3.21 +gabriella thompson 65766 2.71 +gabriella van buren 65581 3.36 +gabriella van buren 65644 2.6 +gabriella white 65638 4.55 +gabriella young 65699 4.13 +gabriella young 65774 3.58 +gabriella zipper 65540 0.96 +gabriella zipper 65754 2.13 +holly allen 65596 0.05 +holly brown 65599 3.23 +holly brown 65619 3.4 +holly falkner 65720 4.06 +holly hernandez 65602 3.67 +holly hernandez 65686 3.39 +holly hernandez 65750 3.2 +holly hernandez 65788 2.95 +holly ichabod 65711 4.73 +holly ichabod 65749 3.54 +holly ichabod 65752 3.27 +holly johnson 65655 4.19 +holly johnson 65662 3.84 +holly johnson 65755 2.65 +holly king 65549 3.61 +holly king 65648 2.37 +holly laertes 65664 4.14 +holly miller 65653 3.84 +holly nixon 65539 4.09 +holly nixon 65658 3.04 +holly polk 65743 2.1 +holly polk 65751 2.78 +holly robinson 65564 0.24 +holly thompson 65538 2.39 +holly thompson 65578 1.49 +holly thompson 65713 1.54 +holly underhill 65634 4.69 +holly underhill 65654 3.08 +holly underhill 65721 3.14 +holly underhill 65759 2.61 +holly van buren 65727 0.04 +holly white 65536 4.59 +holly white 65602 4.13 +holly xylophone 65544 1.49 +holly young 65606 4.39 +holly young 65765 3.81 +holly zipper 65607 4.12 +holly zipper 65755 3.3 +irene allen 65556 3.45 +irene brown 65633 4.8 +irene brown 65650 3.77 +irene brown 65765 3.53 +irene carson 65590 2.83 +irene ellison 65659 3.15 +irene ellison 65696 2.0 +irene falkner 65620 0.99 +irene falkner 65661 1.41 +irene garcia 65660 1.55 +irene garcia 65711 1.72 +irene garcia 65787 1.57 +irene ichabod 65645 0.95 +irene ichabod 65722 1.49 +irene johnson 65583 4.51 +irene laertes 65664 2.5 +irene laertes 65710 1.5 +irene laertes 65722 2.01 +irene miller 65730 4.33 +irene nixon 65631 2.36 +irene nixon 65643 3.43 +irene nixon 65653 2.43 +irene ovid 65691 3.24 +irene ovid 65734 3.17 +irene ovid 65753 3.18 +irene polk 65551 4.51 +irene polk 65575 2.97 +irene polk 65579 3.58 +irene polk 65595 2.82 +irene polk 65610 2.99 +irene quirinius 65724 3.5 +irene quirinius 65769 3.85 +irene quirinius 65773 4.21 +irene robinson 65554 2.67 +irene steinbeck 65683 1.48 +irene thompson 65688 0.06 +irene underhill 65591 3.61 +irene underhill 65787 4.01 +irene van buren 65579 4.26 +irene van buren 65589 4.37 +irene xylophone 65775 4.81 +jessica brown 65588 2.87 +jessica carson 65553 1.02 +jessica carson 65672 1.82 +jessica carson 65747 1.91 +jessica davidson 65549 4.48 +jessica davidson 65606 2.72 +jessica davidson 65675 2.23 +jessica davidson 65727 2.12 +jessica ellison 65567 3.0 +jessica ellison 65663 3.15 +jessica falkner 65584 2.11 +jessica garcia 65676 2.13 +jessica garcia 65789 3.54 +jessica ichabod 65704 3.48 +jessica johnson 65607 3.55 +jessica johnson 65720 3.0 +jessica miller 65733 2.9 +jessica nixon 65590 2.18 +jessica nixon 65774 3.1 +jessica ovid 65582 3.23 +jessica ovid 65751 3.14 +jessica polk 65637 1.12 +jessica quirinius 65562 3.06 +jessica quirinius 65608 1.75 +jessica quirinius 65712 1.51 +jessica quirinius 65716 1.37 +jessica robinson 65576 1.11 +jessica thompson 65581 4.94 +jessica thompson 65675 3.56 +jessica underhill 65656 3.97 +jessica underhill 65702 3.01 +jessica underhill 65783 3.5 +jessica van buren 65615 2.15 +jessica white 65544 1.89 +jessica white 65570 1.92 +jessica white 65594 2.67 +jessica white 65673 2.1 +jessica white 65779 2.04 +jessica xylophone 65562 0.49 +jessica young 65623 0.5 +jessica young 65711 2.37 +jessica zipper 65600 1.71 +jessica zipper 65657 1.0 +jessica zipper 65778 0.9 +katie allen 65542 1.3 +katie brown 65590 3.06 +katie davidson 65619 1.89 +katie ellison 65675 1.92 +katie ellison 65699 2.55 +katie falkner 65728 2.42 +katie garcia 65625 4.18 +katie garcia 65747 4.33 +katie hernandez 65550 2.1 +katie ichabod 65658 1.84 +katie ichabod 65726 2.41 +katie ichabod 65757 3.2 +katie king 65629 0.86 +katie king 65647 2.09 +katie king 65776 2.74 +katie miller 65541 0.66 +katie miller 65661 1.39 +katie nixon 65669 2.06 +katie ovid 65681 1.61 +katie polk 65746 4.74 +katie polk 65784 2.57 +katie robinson 65697 4.74 +katie van buren 65643 0.61 +katie van buren 65730 2.79 +katie white 65620 0.42 +katie white 65719 1.38 +katie xylophone 65585 3.02 +katie young 65644 2.64 +katie young 65746 2.45 +katie young 65764 2.66 +katie zipper 65568 2.33 +katie zipper 65733 1.97 +luke allen 65547 2.04 +luke allen 65552 1.49 +luke allen 65576 2.14 +luke allen 65681 2.2 +luke allen 65776 1.92 +luke brown 65719 4.3 +luke davidson 65656 3.37 +luke davidson 65791 3.9 +luke ellison 65582 0.23 +luke ellison 65664 0.51 +luke ellison 65779 0.87 +luke falkner 65589 2.22 +luke falkner 65618 1.22 +luke garcia 65687 4.73 +luke garcia 65778 3.56 +luke ichabod 65629 3.07 +luke ichabod 65654 3.58 +luke johnson 65545 4.33 +luke johnson 65716 3.01 +luke johnson 65718 3.17 +luke laertes 65608 3.79 +luke laertes 65657 3.89 +luke laertes 65685 2.82 +luke laertes 65730 2.96 +luke laertes 65756 3.19 +luke miller 65752 4.7 +luke ovid 65569 4.4 +luke ovid 65693 2.58 +luke polk 65645 0.57 +luke polk 65658 2.73 +luke quirinius 65655 4.1 +luke robinson 65634 4.9 +luke robinson 65772 4.19 +luke thompson 65626 0.15 +luke underhill 65553 1.28 +luke underhill 65571 0.84 +luke underhill 65651 1.14 +luke van buren 65678 0.34 +luke white 65693 0.91 +luke xylophone 65597 2.27 +luke zipper 65641 4.63 +mike allen 65706 3.06 +mike brown 65654 3.57 +mike carson 65698 4.46 +mike carson 65700 3.89 +mike carson 65751 3.58 +mike davidson 65658 2.06 +mike davidson 65759 3.34 +mike ellison 65598 3.96 +mike ellison 65606 3.28 +mike ellison 65718 3.38 +mike ellison 65738 2.56 +mike ellison 65760 3.03 +mike falkner 65609 4.85 +mike garcia 65571 1.82 +mike garcia 65600 1.42 +mike garcia 65770 1.92 +mike hernandez 65548 1.42 +mike hernandez 65672 1.75 +mike ichabod 65621 3.73 +mike king 65563 4.34 +mike king 65586 3.75 +mike king 65591 3.09 +mike king 65642 2.69 +mike king 65769 2.36 +mike king 65776 2.55 +mike miller 65549 3.96 +mike nixon 65619 0.09 +mike nixon 65704 2.15 +mike polk 65619 4.13 +mike polk 65658 4.27 +mike polk 65704 3.77 +mike quirinius 65717 2.81 +mike steinbeck 65550 3.18 +mike steinbeck 65564 2.58 +mike steinbeck 65573 2.12 +mike steinbeck 65749 1.72 +mike van buren 65620 0.09 +mike van buren 65770 0.88 +mike white 65648 1.72 +mike white 65685 1.12 +mike white 65769 1.74 +mike white 65778 2.05 +mike young 65545 1.69 +mike young 65581 0.92 +mike young 65736 1.84 +mike zipper 65552 4.8 +mike zipper 65695 4.16 +mike zipper 65779 4.22 +nick allen 65641 3.78 +nick allen 65786 3.74 +nick brown 65724 4.61 +nick davidson 65601 0.88 +nick ellison 65691 4.04 +nick ellison 65745 3.22 +nick falkner 65583 4.44 +nick falkner 65676 3.08 +nick garcia 65695 1.05 +nick garcia 65712 2.18 +nick garcia 65720 1.94 +nick ichabod 65572 2.62 +nick ichabod 65681 2.99 +nick ichabod 65737 3.55 +nick johnson 65585 0.56 +nick johnson 65784 0.42 +nick laertes 65624 0.16 +nick miller 65757 4.23 +nick nixon 65650 0.7 +nick ovid 65719 3.93 +nick polk 65716 3.66 +nick quirinius 65588 2.88 +nick quirinius 65723 2.42 +nick robinson 65547 0.21 +nick robinson 65675 0.57 +nick steinbeck 65689 4.11 +nick thompson 65610 2.32 +nick underhill 65619 2.73 +nick van buren 65603 1.84 +nick xylophone 65644 2.06 +nick young 65654 2.06 +nick young 65660 2.28 +nick zipper 65757 3.8 +nick zipper 65765 2.04 +oscar allen 65644 3.01 +oscar brown 65614 3.95 +oscar carson 65537 4.29 +oscar carson 65548 2.89 +oscar carson 65549 1.95 +oscar carson 65624 2.56 +oscar carson 65697 3.03 +oscar davidson 65556 0.6 +oscar ellison 65630 1.47 +oscar ellison 65630 1.47 +oscar falkner 65692 3.44 +oscar garcia 65751 3.71 +oscar hernandez 65683 3.32 +oscar hernandez 65707 2.25 +oscar ichabod 65536 1.8 +oscar ichabod 65562 1.18 +oscar ichabod 65637 1.91 +oscar ichabod 65763 1.96 +oscar johnson 65645 1.69 +oscar johnson 65778 1.59 +oscar king 65541 3.96 +oscar king 65550 4.31 +oscar king 65787 3.54 +oscar laertes 65625 2.26 +oscar laertes 65690 2.55 +oscar laertes 65756 2.85 +oscar laertes 65790 2.39 +oscar nixon 65596 3.12 +oscar ovid 65536 2.94 +oscar ovid 65615 2.95 +oscar ovid 65659 3.41 +oscar polk 65541 1.12 +oscar polk 65643 1.62 +oscar quirinius 65541 3.5 +oscar quirinius 65560 2.87 +oscar quirinius 65689 3.35 +oscar quirinius 65720 3.03 +oscar robinson 65537 0.29 +oscar robinson 65658 0.29 +oscar robinson 65687 1.5 +oscar robinson 65782 1.78 +oscar steinbeck 65709 4.96 +oscar thompson 65542 0.48 +oscar thompson 65698 2.07 +oscar thompson 65727 2.01 +oscar thompson 65738 1.8 +oscar underhill 65626 3.49 +oscar van buren 65581 2.33 +oscar van buren 65635 2.45 +oscar van buren 65705 2.68 +oscar white 65552 3.05 +oscar white 65564 2.58 +oscar white 65671 2.56 +oscar white 65735 2.47 +oscar xylophone 65773 1.51 +oscar xylophone 65773 1.51 +oscar xylophone 65775 1.82 +oscar zipper 65568 4.77 +oscar zipper 65740 3.81 +oscar zipper 65777 3.26 +priscilla brown 65670 2.91 +priscilla brown 65690 2.83 +priscilla brown 65749 2.07 +priscilla carson 65658 1.43 +priscilla carson 65687 2.97 +priscilla carson 65755 2.87 +priscilla ichabod 65627 4.95 +priscilla ichabod 65759 3.41 +priscilla johnson 65543 3.85 +priscilla johnson 65633 2.98 +priscilla johnson 65668 2.23 +priscilla johnson 65681 1.98 +priscilla johnson 65755 1.94 +priscilla king 65646 1.93 +priscilla nixon 65564 0.31 +priscilla nixon 65600 2.35 +priscilla ovid 65541 3.8 +priscilla ovid 65790 2.37 +priscilla polk 65747 3.1 +priscilla quirinius 65672 0.81 +priscilla thompson 65654 2.04 +priscilla underhill 65715 0.82 +priscilla underhill 65729 1.33 +priscilla van buren 65607 0.23 +priscilla van buren 65685 0.81 +priscilla van buren 65749 1.84 +priscilla white 65652 4.79 +priscilla xylophone 65538 3.56 +priscilla xylophone 65763 2.48 +priscilla xylophone 65774 1.84 +priscilla young 65585 2.92 +priscilla young 65658 3.77 +priscilla zipper 65622 4.62 +priscilla zipper 65726 2.67 +quinn allen 65657 3.02 +quinn allen 65708 3.35 +quinn brown 65691 4.37 +quinn brown 65700 3.28 +quinn brown 65733 3.27 +quinn davidson 65549 0.03 +quinn davidson 65714 1.02 +quinn davidson 65776 2.19 +quinn davidson 65779 2.66 +quinn ellison 65705 0.54 +quinn ellison 65778 2.74 +quinn garcia 65568 1.04 +quinn garcia 65604 0.79 +quinn garcia 65610 1.85 +quinn garcia 65773 1.67 +quinn ichabod 65564 0.65 +quinn king 65558 2.38 +quinn king 65649 1.53 +quinn laertes 65542 2.33 +quinn laertes 65560 2.03 +quinn laertes 65627 2.08 +quinn nixon 65659 0.72 +quinn ovid 65699 2.87 +quinn quirinius 65747 0.86 +quinn robinson 65627 4.14 +quinn steinbeck 65578 4.92 +quinn steinbeck 65763 3.18 +quinn thompson 65643 4.62 +quinn thompson 65774 2.43 +quinn underhill 65549 3.89 +quinn underhill 65694 2.39 +quinn underhill 65767 2.48 +quinn van buren 65725 0.53 +quinn young 65647 3.54 +quinn zipper 65579 1.83 +quinn zipper 65693 2.1 +rachel allen 65661 4.66 +rachel allen 65709 4.53 +rachel brown 65586 0.43 +rachel brown 65587 2.69 +rachel brown 65587 2.69 +rachel brown 65610 2.67 +rachel brown 65693 3.07 +rachel carson 65677 4.58 +rachel carson 65682 4.54 +rachel davidson 65755 2.28 +rachel ellison 65761 0.05 +rachel falkner 65616 1.2 +rachel falkner 65681 2.01 +rachel falkner 65693 2.56 +rachel falkner 65764 2.91 +rachel johnson 65658 3.02 +rachel king 65604 2.32 +rachel king 65643 2.4 +rachel laertes 65562 4.43 +rachel laertes 65624 2.8 +rachel ovid 65721 0.3 +rachel ovid 65736 0.92 +rachel polk 65686 2.56 +rachel quirinius 65787 2.95 +rachel robinson 65544 0.06 +rachel robinson 65717 1.55 +rachel robinson 65724 2.27 +rachel thompson 65648 2.49 +rachel thompson 65662 3.16 +rachel thompson 65733 2.51 +rachel underhill 65667 2.29 +rachel white 65615 1.99 +rachel white 65717 3.08 +rachel young 65727 1.75 +rachel zipper 65757 2.82 +rachel zipper 65785 3.62 +sarah carson 65679 1.04 +sarah carson 65693 0.85 +sarah carson 65694 2.06 +sarah ellison 65611 1.64 +sarah falkner 65606 0.77 +sarah falkner 65680 2.85 +sarah garcia 65563 4.89 +sarah garcia 65638 4.78 +sarah garcia 65661 3.63 +sarah ichabod 65667 3.94 +sarah ichabod 65671 2.33 +sarah johnson 65659 3.51 +sarah johnson 65716 4.21 +sarah johnson 65731 3.81 +sarah johnson 65751 3.37 +sarah king 65650 1.05 +sarah king 65699 0.99 +sarah miller 65557 0.2 +sarah ovid 65550 3.21 +sarah robinson 65677 4.9 +sarah robinson 65763 2.99 +sarah steinbeck 65721 2.82 +sarah white 65622 0.07 +sarah white 65747 2.29 +sarah xylophone 65678 0.15 +sarah young 65595 2.15 +sarah zipper 65550 2.22 +tom brown 65593 1.64 +tom brown 65675 2.83 +tom carson 65539 4.38 +tom carson 65624 4.28 +tom carson 65780 4.03 +tom davidson 65780 2.4 +tom ellison 65578 3.3 +tom ellison 65670 4.04 +tom ellison 65756 3.51 +tom falkner 65574 1.09 +tom falkner 65583 2.05 +tom hernandez 65575 2.35 +tom hernandez 65632 2.64 +tom ichabod 65588 1.48 +tom johnson 65536 4.68 +tom johnson 65789 4.6 +tom king 65576 2.87 +tom laertes 65617 1.51 +tom laertes 65701 1.93 +tom miller 65594 1.14 +tom miller 65603 1.25 +tom miller 65704 1.26 +tom nixon 65672 0.04 +tom ovid 65628 1.95 +tom polk 65652 2.54 +tom polk 65742 2.26 +tom quirinius 65563 4.51 +tom quirinius 65783 4.17 +tom robinson 65626 3.12 +tom robinson 65632 3.61 +tom robinson 65691 3.13 +tom robinson 65758 3.45 +tom steinbeck 65666 1.34 +tom van buren 65621 0.66 +tom van buren 65652 2.71 +tom van buren 65669 3.47 +tom white 65548 2.13 +tom young 65544 3.73 +tom young 65546 2.02 +tom zipper 65789 1.51 +ulysses brown 65735 2.92 +ulysses carson 65602 0.34 +ulysses carson 65643 2.11 +ulysses carson 65703 1.43 +ulysses carson 65716 1.41 +ulysses davidson 65750 3.04 +ulysses ellison 65575 4.39 +ulysses garcia 65666 4.2 +ulysses hernandez 65651 1.75 +ulysses hernandez 65702 2.42 +ulysses hernandez 65786 2.53 +ulysses ichabod 65551 0.33 +ulysses ichabod 65566 2.19 +ulysses johnson 65776 4.79 +ulysses king 65649 4.46 +ulysses laertes 65691 4.55 +ulysses laertes 65711 3.54 +ulysses laertes 65781 3.66 +ulysses miller 65610 0.24 +ulysses miller 65637 1.08 +ulysses nixon 65603 1.85 +ulysses ovid 65656 3.17 +ulysses polk 65563 1.32 +ulysses polk 65580 3.05 +ulysses polk 65612 3.46 +ulysses polk 65777 3.75 +ulysses quirinius 65786 2.13 +ulysses robinson 65744 1.97 +ulysses steinbeck 65611 2.74 +ulysses steinbeck 65680 2.64 +ulysses thompson 65788 1.51 +ulysses underhill 65570 0.38 +ulysses underhill 65616 0.8 +ulysses underhill 65620 2.09 +ulysses underhill 65623 2.69 +ulysses underhill 65641 2.54 +ulysses underhill 65713 2.9 +ulysses underhill 65785 2.97 +ulysses van buren 65684 1.42 +ulysses white 65654 0.14 +ulysses white 65675 1.51 +ulysses xylophone 65623 2.3 +ulysses xylophone 65636 2.69 +ulysses xylophone 65781 3.22 +ulysses young 65675 1.34 +ulysses young 65736 2.01 +ulysses young 65748 2.24 +victor allen 65684 0.83 +victor allen 65707 2.31 +victor brown 65550 4.57 +victor brown 65555 3.54 +victor brown 65622 2.61 +victor brown 65673 2.34 +victor davidson 65579 0.61 +victor davidson 65628 1.52 +victor davidson 65783 2.25 +victor ellison 65641 1.32 +victor ellison 65782 2.59 +victor hernandez 65571 3.62 +victor hernandez 65659 3.68 +victor hernandez 65708 3.35 +victor hernandez 65735 2.88 +victor hernandez 65775 2.62 +victor johnson 65606 3.03 +victor johnson 65607 2.3 +victor johnson 65607 2.3 +victor king 65721 4.09 +victor king 65743 2.45 +victor laertes 65638 1.46 +victor laertes 65644 2.38 +victor miller 65570 0.1 +victor nixon 65709 0.74 +victor nixon 65791 1.73 +victor ovid 65649 4.93 +victor polk 65625 1.04 +victor quirinius 65620 1.32 +victor quirinius 65651 3.15 +victor robinson 65596 0.92 +victor robinson 65673 1.76 +victor steinbeck 65618 2.87 +victor steinbeck 65661 2.19 +victor steinbeck 65686 2.81 +victor thompson 65548 1.59 +victor van buren 65664 4.44 +victor van buren 65774 4.06 +victor white 65548 4.67 +victor white 65601 3.87 +victor xylophone 65549 3.8 +victor xylophone 65618 2.13 +victor xylophone 65644 1.59 +victor xylophone 65677 1.89 +victor xylophone 65755 2.27 +victor young 65628 3.16 +victor zipper 65743 3.98 +wendy allen 65628 3.8 +wendy allen 65711 3.44 +wendy allen 65782 2.4 +wendy brown 65580 4.67 +wendy brown 65657 4.68 +wendy ellison 65545 1.51 +wendy ellison 65603 1.6 +wendy falkner 65595 0.58 +wendy falkner 65604 0.82 +wendy falkner 65635 1.59 +wendy garcia 65659 3.47 +wendy garcia 65746 2.35 +wendy garcia 65747 1.67 +wendy garcia 65777 1.32 +wendy hernandez 65650 2.26 +wendy ichabod 65730 0.44 +wendy king 65586 4.46 +wendy king 65664 4.25 +wendy king 65670 2.94 +wendy laertes 65566 3.13 +wendy laertes 65683 3.99 +wendy laertes 65727 3.57 +wendy miller 65582 1.53 +wendy miller 65626 1.4 +wendy nixon 65611 0.26 +wendy nixon 65746 2.27 +wendy ovid 65589 4.75 +wendy ovid 65643 3.42 +wendy polk 65656 0.62 +wendy polk 65692 1.36 +wendy quirinius 65766 1.35 +wendy quirinius 65767 2.76 +wendy robinson 65622 0.85 +wendy robinson 65715 2.13 +wendy robinson 65774 1.8 +wendy steinbeck 65612 0.07 +wendy thompson 65650 2.27 +wendy thompson 65737 3.2 +wendy underhill 65662 4.55 +wendy underhill 65758 2.84 +wendy underhill 65775 2.54 +wendy van buren 65680 1.1 +wendy van buren 65699 1.0 +wendy white 65705 0.5 +wendy xylophone 65687 0.46 +wendy xylophone 65773 1.39 +wendy young 65674 0.48 +wendy young 65685 1.31 +xavier allen 65611 1.53 +xavier allen 65618 2.07 +xavier allen 65771 2.33 +xavier brown 65600 0.89 +xavier brown 65704 0.58 +xavier brown 65723 1.55 +xavier carson 65731 4.42 +xavier carson 65758 3.91 +xavier davidson 65644 1.84 +xavier davidson 65664 3.4 +xavier davidson 65755 2.67 +xavier ellison 65541 1.47 +xavier ellison 65654 2.49 +xavier garcia 65672 2.76 +xavier hernandez 65541 0.96 +xavier hernandez 65544 1.47 +xavier hernandez 65766 1.28 +xavier ichabod 65597 4.76 +xavier ichabod 65663 2.99 +xavier johnson 65654 1.34 +xavier johnson 65744 3.06 +xavier king 65590 2.68 +xavier king 65601 1.4 +xavier laertes 65743 0.75 +xavier ovid 65788 2.06 +xavier polk 65587 0.99 +xavier polk 65653 1.15 +xavier polk 65675 1.9 +xavier polk 65696 1.93 +xavier quirinius 65599 1.66 +xavier quirinius 65650 1.94 +xavier quirinius 65656 2.46 +xavier quirinius 65737 1.92 +xavier thompson 65608 1.65 +xavier underhill 65710 0.13 +xavier white 65703 0.98 +xavier white 65732 2.22 +xavier xylophone 65572 1.0 +xavier zipper 65561 0.94 +yuri allen 65565 2.03 +yuri allen 65682 1.14 +yuri brown 65538 2.73 +yuri brown 65688 2.02 +yuri carson 65670 3.06 +yuri carson 65769 3.43 +yuri ellison 65570 1.05 +yuri ellison 65581 1.68 +yuri falkner 65658 2.85 +yuri falkner 65681 2.14 +yuri garcia 65639 3.41 +yuri hernandez 65706 1.64 +yuri johnson 65587 1.27 +yuri johnson 65697 1.44 +yuri johnson 65712 2.29 +yuri king 65721 0.33 +yuri laertes 65637 4.3 +yuri laertes 65773 2.15 +yuri nixon 65635 4.02 +yuri nixon 65740 4.18 +yuri polk 65607 0.08 +yuri polk 65713 0.37 +yuri polk 65742 1.25 +yuri quirinius 65544 2.58 +yuri quirinius 65617 2.1 +yuri quirinius 65695 1.91 +yuri steinbeck 65592 4.89 +yuri steinbeck 65679 3.24 +yuri thompson 65676 2.67 +yuri underhill 65718 2.86 +yuri underhill 65750 2.51 +yuri white 65659 4.59 +yuri xylophone 65714 2.53 +zach allen 65667 0.88 +zach brown 65559 4.88 +zach brown 65588 4.53 +zach brown 65691 3.49 +zach brown 65759 3.4 +zach brown 65762 3.55 +zach carson 65572 2.03 +zach ellison 65748 1.76 +zach falkner 65620 0.34 +zach falkner 65627 0.25 +zach garcia 65544 0.99 +zach garcia 65623 2.84 +zach garcia 65629 3.01 +zach garcia 65786 2.55 +zach ichabod 65599 3.36 +zach ichabod 65612 1.92 +zach king 65556 2.36 +zach king 65702 1.52 +zach king 65773 2.58 +zach miller 65584 1.6 +zach miller 65665 0.99 +zach miller 65719 1.55 +zach ovid 65578 1.51 +zach ovid 65703 1.92 +zach ovid 65750 2.63 +zach ovid 65784 2.72 +zach quirinius 65691 2.95 +zach robinson 65599 2.87 +zach steinbeck 65602 2.45 +zach steinbeck 65695 1.86 +zach thompson 65636 0.25 +zach thompson 65696 0.51 +zach underhill 65573 3.97 +zach white 65733 2.31 +zach xylophone 65542 1.69 +zach xylophone 65780 0.88 +zach young 65576 1.82 +zach zipper 65579 4.5 +zach zipper 65649 4.02 +zach zipper 65676 3.12 diff --git a/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-8-45a1d7c2aba45d761e19ff4dfdf5463e b/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-8-45a1d7c2aba45d761e19ff4dfdf5463e new file mode 100644 index 0000000000000..84b934fad85b1 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-8-45a1d7c2aba45d761e19ff4dfdf5463e @@ -0,0 +1,1049 @@ + 65560 20.0 + 65718 20.0 + 65740 20.0 +alice allen 65662 20.0 +alice allen 65720 20.0 +alice allen 65758 20.0 +alice brown 65696 20.0 +alice carson 65559 20.0 +alice davidson 65547 20.0 +alice falkner 65669 20.0 +alice garcia 65613 20.0 +alice hernandez 65737 20.0 +alice hernandez 65784 20.0 +alice johnson 65739 20.0 +alice king 65660 20.0 +alice king 65734 20.0 +alice king 65738 20.0 +alice laertes 65669 20.0 +alice laertes 65671 20.0 +alice miller 65590 20.0 +alice nixon 65586 20.0 +alice nixon 65595 20.0 +alice nixon 65604 20.0 +alice ovid 65737 20.0 +alice polk 65548 20.0 +alice quirinius 65636 20.0 +alice quirinius 65728 20.0 +alice robinson 65606 20.0 +alice robinson 65789 20.0 +alice steinbeck 65578 20.0 +alice steinbeck 65673 20.0 +alice steinbeck 65786 20.0 +alice underhill 65750 20.0 +alice van buren 65562 20.0 +alice xylophone 65578 20.0 +alice xylophone 65585 20.0 +alice xylophone 65599 20.0 +alice zipper 65553 20.0 +alice zipper 65662 20.0 +alice zipper 65766 20.0 +bob brown 65584 20.0 +bob brown 65777 20.0 +bob brown 65783 20.0 +bob carson 65713 20.0 +bob davidson 65664 20.0 +bob davidson 65693 20.0 +bob davidson 65768 20.0 +bob ellison 65591 20.0 +bob ellison 65624 20.0 +bob ellison 65721 20.0 +bob ellison 65760 20.0 +bob falkner 65789 20.0 +bob garcia 65585 20.0 +bob garcia 65598 20.0 +bob garcia 65673 20.0 +bob garcia 65754 20.0 +bob garcia 65782 20.0 +bob hernandez 65557 20.0 +bob ichabod 65549 20.0 +bob king 65715 20.0 +bob king 65757 20.0 +bob king 65783 20.0 +bob laertes 65602 20.0 +bob laertes 65663 20.0 +bob miller 65608 20.0 +bob ovid 65564 20.0 +bob ovid 65619 20.0 +bob ovid 65686 20.0 +bob ovid 65726 20.0 +bob polk 65594 20.0 +bob quirinius 65700 20.0 +bob steinbeck 65637 20.0 +bob van buren 65778 20.0 +bob white 65543 20.0 +bob white 65605 20.0 +bob xylophone 65574 20.0 +bob xylophone 65666 20.0 +bob young 65556 20.0 +bob zipper 65559 20.0 +bob zipper 65633 20.0 +bob zipper 65739 20.0 +calvin allen 65669 20.0 +calvin brown 65537 20.0 +calvin brown 65580 20.0 +calvin brown 65677 20.0 +calvin carson 65637 20.0 +calvin davidson 65541 20.0 +calvin davidson 65564 20.0 +calvin ellison 65667 20.0 +calvin falkner 65573 20.0 +calvin falkner 65596 20.0 +calvin falkner 65738 20.0 +calvin falkner 65762 20.0 +calvin falkner 65778 20.0 +calvin falkner 65784 20.0 +calvin garcia 65664 20.0 +calvin hernandez 65578 20.0 +calvin johnson 65731 20.0 +calvin laertes 65570 20.0 +calvin laertes 65684 20.0 +calvin nixon 65654 20.0 +calvin nixon 65724 20.0 +calvin nixon 65749 20.0 +calvin ovid 65554 20.0 +calvin ovid 65643 20.0 +calvin ovid 65663 20.0 +calvin ovid 65715 20.0 +calvin polk 65731 20.0 +calvin quirinius 65741 20.0 +calvin quirinius 65769 20.0 +calvin robinson 65581 20.0 +calvin steinbeck 65680 20.0 +calvin steinbeck 65762 20.0 +calvin steinbeck 65779 20.0 +calvin thompson 65560 20.0 +calvin thompson 65640 20.0 +calvin underhill 65732 20.0 +calvin van buren 65552 20.0 +calvin van buren 65771 20.0 +calvin white 65553 20.0 +calvin white 65561 20.0 +calvin xylophone 65575 20.0 +calvin xylophone 65596 20.0 +calvin xylophone 65713 20.0 +calvin young 65574 20.0 +calvin young 65746 20.0 +calvin zipper 65669 20.0 +calvin zipper 65739 20.0 +david allen 65588 20.0 +david allen 65617 20.0 +david brown 65637 20.0 +david brown 65760 20.0 +david davidson 65559 20.0 +david davidson 65756 20.0 +david davidson 65778 20.0 +david davidson 65779 20.0 +david ellison 65634 20.0 +david ellison 65724 20.0 +david ellison 65724 20.0 +david hernandez 65763 20.0 +david ichabod 65699 20.0 +david ichabod 65715 20.0 +david laertes 65762 20.0 +david nixon 65536 20.0 +david ovid 65623 20.0 +david ovid 65628 20.0 +david quirinius 65697 20.0 +david quirinius 65759 20.0 +david quirinius 65779 20.0 +david robinson 65762 20.0 +david robinson 65775 20.0 +david thompson 65550 20.0 +david underhill 65602 20.0 +david underhill 65662 20.0 +david underhill 65751 20.0 +david van buren 65625 20.0 +david van buren 65634 20.0 +david white 65678 20.0 +david xylophone 65537 20.0 +david xylophone 65670 20.0 +david xylophone 65764 20.0 +david young 65551 20.0 +david young 65694 20.0 +ethan allen 65747 20.0 +ethan brown 65539 20.0 +ethan brown 65617 20.0 +ethan brown 65685 20.0 +ethan brown 65685 20.0 +ethan brown 65722 20.0 +ethan brown 65733 20.0 +ethan carson 65742 20.0 +ethan ellison 65714 20.0 +ethan ellison 65732 20.0 +ethan falkner 65577 20.0 +ethan falkner 65614 20.0 +ethan garcia 65736 20.0 +ethan hernandez 65618 20.0 +ethan johnson 65536 20.0 +ethan king 65614 20.0 +ethan laertes 65562 20.0 +ethan laertes 65597 20.0 +ethan laertes 65628 20.0 +ethan laertes 65643 20.0 +ethan laertes 65680 20.0 +ethan laertes 65745 20.0 +ethan laertes 65760 20.0 +ethan miller 65712 20.0 +ethan nixon 65766 20.0 +ethan ovid 65697 20.0 +ethan polk 65589 20.0 +ethan polk 65615 20.0 +ethan polk 65622 20.0 +ethan polk 65622 20.0 +ethan quirinius 65542 20.0 +ethan quirinius 65591 20.0 +ethan quirinius 65706 20.0 +ethan robinson 65547 20.0 +ethan robinson 65659 20.0 +ethan underhill 65570 20.0 +ethan van buren 65572 20.0 +ethan white 65677 20.0 +ethan white 65788 20.0 +ethan xylophone 65595 20.0 +ethan zipper 65593 20.0 +ethan zipper 65680 20.0 +fred davidson 65552 20.0 +fred davidson 65595 20.0 +fred davidson 65721 20.0 +fred ellison 65548 20.0 +fred ellison 65691 20.0 +fred ellison 65771 20.0 +fred falkner 65637 20.0 +fred falkner 65648 20.0 +fred falkner 65783 20.0 +fred hernandez 65541 20.0 +fred ichabod 65572 20.0 +fred ichabod 65789 20.0 +fred johnson 65758 20.0 +fred king 65694 20.0 +fred king 65745 20.0 +fred laertes 65769 20.0 +fred miller 65536 20.0 +fred nixon 65560 20.0 +fred nixon 65612 20.0 +fred nixon 65703 20.0 +fred nixon 65705 20.0 +fred polk 65603 20.0 +fred polk 65656 20.0 +fred polk 65701 20.0 +fred polk 65706 20.0 +fred quirinius 65697 20.0 +fred quirinius 65782 20.0 +fred robinson 65623 20.0 +fred steinbeck 65544 20.0 +fred steinbeck 65651 20.0 +fred steinbeck 65755 20.0 +fred underhill 65629 20.0 +fred van buren 65537 20.0 +fred van buren 65561 20.0 +fred van buren 65745 20.0 +fred van buren 65789 20.0 +fred white 65589 20.0 +fred young 65594 20.0 +fred young 65773 20.0 +fred zipper 65553 20.0 +gabriella allen 65646 20.0 +gabriella allen 65677 20.0 +gabriella brown 65704 20.0 +gabriella brown 65753 20.0 +gabriella carson 65586 20.0 +gabriella davidson 65565 20.0 +gabriella ellison 65706 20.0 +gabriella ellison 65716 20.0 +gabriella falkner 65623 20.0 +gabriella falkner 65711 20.0 +gabriella falkner 65767 20.0 +gabriella garcia 65571 20.0 +gabriella hernandez 65587 20.0 +gabriella hernandez 65717 20.0 +gabriella ichabod 65559 20.0 +gabriella ichabod 65633 20.0 +gabriella ichabod 65702 20.0 +gabriella ichabod 65712 20.0 +gabriella ichabod 65717 20.0 +gabriella king 65651 20.0 +gabriella king 65657 20.0 +gabriella laertes 65781 20.0 +gabriella miller 65646 20.0 +gabriella ovid 65556 20.0 +gabriella ovid 65583 20.0 +gabriella polk 65701 20.0 +gabriella polk 65790 20.0 +gabriella steinbeck 65582 20.0 +gabriella steinbeck 65653 20.0 +gabriella thompson 65682 20.0 +gabriella thompson 65755 20.0 +gabriella thompson 65766 20.0 +gabriella van buren 65581 20.0 +gabriella van buren 65644 20.0 +gabriella white 65638 20.0 +gabriella young 65699 20.0 +gabriella young 65774 20.0 +gabriella zipper 65540 20.0 +gabriella zipper 65754 20.0 +holly allen 65596 20.0 +holly brown 65599 20.0 +holly brown 65619 20.0 +holly falkner 65720 20.0 +holly hernandez 65602 20.0 +holly hernandez 65686 20.0 +holly hernandez 65750 20.0 +holly hernandez 65788 20.0 +holly ichabod 65711 20.0 +holly ichabod 65749 20.0 +holly ichabod 65752 20.0 +holly johnson 65655 20.0 +holly johnson 65662 20.0 +holly johnson 65755 20.0 +holly king 65549 20.0 +holly king 65648 20.0 +holly laertes 65664 20.0 +holly miller 65653 20.0 +holly nixon 65539 20.0 +holly nixon 65658 20.0 +holly polk 65743 20.0 +holly polk 65751 20.0 +holly robinson 65564 20.0 +holly thompson 65538 20.0 +holly thompson 65578 20.0 +holly thompson 65713 20.0 +holly underhill 65634 20.0 +holly underhill 65654 20.0 +holly underhill 65721 20.0 +holly underhill 65759 20.0 +holly van buren 65727 20.0 +holly white 65536 20.0 +holly white 65602 20.0 +holly xylophone 65544 20.0 +holly young 65606 20.0 +holly young 65765 20.0 +holly zipper 65607 20.0 +holly zipper 65755 20.0 +irene allen 65556 20.0 +irene brown 65633 20.0 +irene brown 65650 20.0 +irene brown 65765 20.0 +irene carson 65590 20.0 +irene ellison 65659 20.0 +irene ellison 65696 20.0 +irene falkner 65620 20.0 +irene falkner 65661 20.0 +irene garcia 65660 20.0 +irene garcia 65711 20.0 +irene garcia 65787 20.0 +irene ichabod 65645 20.0 +irene ichabod 65722 20.0 +irene johnson 65583 20.0 +irene laertes 65664 20.0 +irene laertes 65710 20.0 +irene laertes 65722 20.0 +irene miller 65730 20.0 +irene nixon 65631 20.0 +irene nixon 65643 20.0 +irene nixon 65653 20.0 +irene ovid 65691 20.0 +irene ovid 65734 20.0 +irene ovid 65753 20.0 +irene polk 65551 20.0 +irene polk 65575 20.0 +irene polk 65579 20.0 +irene polk 65595 20.0 +irene polk 65610 20.0 +irene quirinius 65724 20.0 +irene quirinius 65769 20.0 +irene quirinius 65773 20.0 +irene robinson 65554 20.0 +irene steinbeck 65683 20.0 +irene thompson 65688 20.0 +irene underhill 65591 20.0 +irene underhill 65787 20.0 +irene van buren 65579 20.0 +irene van buren 65589 20.0 +irene xylophone 65775 20.0 +jessica brown 65588 20.0 +jessica carson 65553 20.0 +jessica carson 65672 20.0 +jessica carson 65747 20.0 +jessica davidson 65549 20.0 +jessica davidson 65606 20.0 +jessica davidson 65675 20.0 +jessica davidson 65727 20.0 +jessica ellison 65567 20.0 +jessica ellison 65663 20.0 +jessica falkner 65584 20.0 +jessica garcia 65676 20.0 +jessica garcia 65789 20.0 +jessica ichabod 65704 20.0 +jessica johnson 65607 20.0 +jessica johnson 65720 20.0 +jessica miller 65733 20.0 +jessica nixon 65590 20.0 +jessica nixon 65774 20.0 +jessica ovid 65582 20.0 +jessica ovid 65751 20.0 +jessica polk 65637 20.0 +jessica quirinius 65562 20.0 +jessica quirinius 65608 20.0 +jessica quirinius 65712 20.0 +jessica quirinius 65716 20.0 +jessica robinson 65576 20.0 +jessica thompson 65581 20.0 +jessica thompson 65675 20.0 +jessica underhill 65656 20.0 +jessica underhill 65702 20.0 +jessica underhill 65783 20.0 +jessica van buren 65615 20.0 +jessica white 65544 20.0 +jessica white 65570 20.0 +jessica white 65594 20.0 +jessica white 65673 20.0 +jessica white 65779 20.0 +jessica xylophone 65562 20.0 +jessica young 65623 20.0 +jessica young 65711 20.0 +jessica zipper 65600 20.0 +jessica zipper 65657 20.0 +jessica zipper 65778 20.0 +katie allen 65542 20.0 +katie brown 65590 20.0 +katie davidson 65619 20.0 +katie ellison 65675 20.0 +katie ellison 65699 20.0 +katie falkner 65728 20.0 +katie garcia 65625 20.0 +katie garcia 65747 20.0 +katie hernandez 65550 20.0 +katie ichabod 65658 20.0 +katie ichabod 65726 20.0 +katie ichabod 65757 20.0 +katie king 65629 20.0 +katie king 65647 20.0 +katie king 65776 20.0 +katie miller 65541 20.0 +katie miller 65661 20.0 +katie nixon 65669 20.0 +katie ovid 65681 20.0 +katie polk 65746 20.0 +katie polk 65784 20.0 +katie robinson 65697 20.0 +katie van buren 65643 20.0 +katie van buren 65730 20.0 +katie white 65620 20.0 +katie white 65719 20.0 +katie xylophone 65585 20.0 +katie young 65644 20.0 +katie young 65746 20.0 +katie young 65764 20.0 +katie zipper 65568 20.0 +katie zipper 65733 20.0 +luke allen 65547 20.0 +luke allen 65552 20.0 +luke allen 65576 20.0 +luke allen 65681 20.0 +luke allen 65776 20.0 +luke brown 65719 20.0 +luke davidson 65656 20.0 +luke davidson 65791 20.0 +luke ellison 65582 20.0 +luke ellison 65664 20.0 +luke ellison 65779 20.0 +luke falkner 65589 20.0 +luke falkner 65618 20.0 +luke garcia 65687 20.0 +luke garcia 65778 20.0 +luke ichabod 65629 20.0 +luke ichabod 65654 20.0 +luke johnson 65545 20.0 +luke johnson 65716 20.0 +luke johnson 65718 20.0 +luke laertes 65608 20.0 +luke laertes 65657 20.0 +luke laertes 65685 20.0 +luke laertes 65730 20.0 +luke laertes 65756 20.0 +luke miller 65752 20.0 +luke ovid 65569 20.0 +luke ovid 65693 20.0 +luke polk 65645 20.0 +luke polk 65658 20.0 +luke quirinius 65655 20.0 +luke robinson 65634 20.0 +luke robinson 65772 20.0 +luke thompson 65626 20.0 +luke underhill 65553 20.0 +luke underhill 65571 20.0 +luke underhill 65651 20.0 +luke van buren 65678 20.0 +luke white 65693 20.0 +luke xylophone 65597 20.0 +luke zipper 65641 20.0 +mike allen 65706 20.0 +mike brown 65654 20.0 +mike carson 65698 20.0 +mike carson 65700 20.0 +mike carson 65751 20.0 +mike davidson 65658 20.0 +mike davidson 65759 20.0 +mike ellison 65598 20.0 +mike ellison 65606 20.0 +mike ellison 65718 20.0 +mike ellison 65738 20.0 +mike ellison 65760 20.0 +mike falkner 65609 20.0 +mike garcia 65571 20.0 +mike garcia 65600 20.0 +mike garcia 65770 20.0 +mike hernandez 65548 20.0 +mike hernandez 65672 20.0 +mike ichabod 65621 20.0 +mike king 65563 20.0 +mike king 65586 20.0 +mike king 65591 20.0 +mike king 65642 20.0 +mike king 65769 20.0 +mike king 65776 20.0 +mike miller 65549 20.0 +mike nixon 65619 20.0 +mike nixon 65704 20.0 +mike polk 65619 20.0 +mike polk 65658 20.0 +mike polk 65704 20.0 +mike quirinius 65717 20.0 +mike steinbeck 65550 20.0 +mike steinbeck 65564 20.0 +mike steinbeck 65573 20.0 +mike steinbeck 65749 20.0 +mike van buren 65620 20.0 +mike van buren 65770 20.0 +mike white 65648 20.0 +mike white 65685 20.0 +mike white 65769 20.0 +mike white 65778 20.0 +mike young 65545 20.0 +mike young 65581 20.0 +mike young 65736 20.0 +mike zipper 65552 20.0 +mike zipper 65695 20.0 +mike zipper 65779 20.0 +nick allen 65641 20.0 +nick allen 65786 20.0 +nick brown 65724 20.0 +nick davidson 65601 20.0 +nick ellison 65691 20.0 +nick ellison 65745 20.0 +nick falkner 65583 20.0 +nick falkner 65676 20.0 +nick garcia 65695 20.0 +nick garcia 65712 20.0 +nick garcia 65720 20.0 +nick ichabod 65572 20.0 +nick ichabod 65681 20.0 +nick ichabod 65737 20.0 +nick johnson 65585 20.0 +nick johnson 65784 20.0 +nick laertes 65624 20.0 +nick miller 65757 20.0 +nick nixon 65650 20.0 +nick ovid 65719 20.0 +nick polk 65716 20.0 +nick quirinius 65588 20.0 +nick quirinius 65723 20.0 +nick robinson 65547 20.0 +nick robinson 65675 20.0 +nick steinbeck 65689 20.0 +nick thompson 65610 20.0 +nick underhill 65619 20.0 +nick van buren 65603 20.0 +nick xylophone 65644 20.0 +nick young 65654 20.0 +nick young 65660 20.0 +nick zipper 65757 20.0 +nick zipper 65765 20.0 +oscar allen 65644 20.0 +oscar brown 65614 20.0 +oscar carson 65537 20.0 +oscar carson 65548 20.0 +oscar carson 65549 20.0 +oscar carson 65624 20.0 +oscar carson 65697 20.0 +oscar davidson 65556 20.0 +oscar ellison 65630 20.0 +oscar ellison 65630 20.0 +oscar falkner 65692 20.0 +oscar garcia 65751 20.0 +oscar hernandez 65683 20.0 +oscar hernandez 65707 20.0 +oscar ichabod 65536 20.0 +oscar ichabod 65562 20.0 +oscar ichabod 65637 20.0 +oscar ichabod 65763 20.0 +oscar johnson 65645 20.0 +oscar johnson 65778 20.0 +oscar king 65541 20.0 +oscar king 65550 20.0 +oscar king 65787 20.0 +oscar laertes 65625 20.0 +oscar laertes 65690 20.0 +oscar laertes 65756 20.0 +oscar laertes 65790 20.0 +oscar nixon 65596 20.0 +oscar ovid 65536 20.0 +oscar ovid 65615 20.0 +oscar ovid 65659 20.0 +oscar polk 65541 20.0 +oscar polk 65643 20.0 +oscar quirinius 65541 20.0 +oscar quirinius 65560 20.0 +oscar quirinius 65689 20.0 +oscar quirinius 65720 20.0 +oscar robinson 65537 20.0 +oscar robinson 65658 20.0 +oscar robinson 65687 20.0 +oscar robinson 65782 20.0 +oscar steinbeck 65709 20.0 +oscar thompson 65542 20.0 +oscar thompson 65698 20.0 +oscar thompson 65727 20.0 +oscar thompson 65738 20.0 +oscar underhill 65626 20.0 +oscar van buren 65581 20.0 +oscar van buren 65635 20.0 +oscar van buren 65705 20.0 +oscar white 65552 20.0 +oscar white 65564 20.0 +oscar white 65671 20.0 +oscar white 65735 20.0 +oscar xylophone 65773 20.0 +oscar xylophone 65773 20.0 +oscar xylophone 65775 20.0 +oscar zipper 65568 20.0 +oscar zipper 65740 20.0 +oscar zipper 65777 20.0 +priscilla brown 65670 20.0 +priscilla brown 65690 20.0 +priscilla brown 65749 20.0 +priscilla carson 65658 20.0 +priscilla carson 65687 20.0 +priscilla carson 65755 20.0 +priscilla ichabod 65627 20.0 +priscilla ichabod 65759 20.0 +priscilla johnson 65543 20.0 +priscilla johnson 65633 20.0 +priscilla johnson 65668 20.0 +priscilla johnson 65681 20.0 +priscilla johnson 65755 20.0 +priscilla king 65646 20.0 +priscilla nixon 65564 20.0 +priscilla nixon 65600 20.0 +priscilla ovid 65541 20.0 +priscilla ovid 65790 20.0 +priscilla polk 65747 20.0 +priscilla quirinius 65672 20.0 +priscilla thompson 65654 20.0 +priscilla underhill 65715 20.0 +priscilla underhill 65729 20.0 +priscilla van buren 65607 20.0 +priscilla van buren 65685 20.0 +priscilla van buren 65749 20.0 +priscilla white 65652 20.0 +priscilla xylophone 65538 20.0 +priscilla xylophone 65763 20.0 +priscilla xylophone 65774 20.0 +priscilla young 65585 20.0 +priscilla young 65658 20.0 +priscilla zipper 65622 20.0 +priscilla zipper 65726 20.0 +quinn allen 65657 20.0 +quinn allen 65708 20.0 +quinn brown 65691 20.0 +quinn brown 65700 20.0 +quinn brown 65733 20.0 +quinn davidson 65549 20.0 +quinn davidson 65714 20.0 +quinn davidson 65776 20.0 +quinn davidson 65779 20.0 +quinn ellison 65705 20.0 +quinn ellison 65778 20.0 +quinn garcia 65568 20.0 +quinn garcia 65604 20.0 +quinn garcia 65610 20.0 +quinn garcia 65773 20.0 +quinn ichabod 65564 20.0 +quinn king 65558 20.0 +quinn king 65649 20.0 +quinn laertes 65542 20.0 +quinn laertes 65560 20.0 +quinn laertes 65627 20.0 +quinn nixon 65659 20.0 +quinn ovid 65699 20.0 +quinn quirinius 65747 20.0 +quinn robinson 65627 20.0 +quinn steinbeck 65578 20.0 +quinn steinbeck 65763 20.0 +quinn thompson 65643 20.0 +quinn thompson 65774 20.0 +quinn underhill 65549 20.0 +quinn underhill 65694 20.0 +quinn underhill 65767 20.0 +quinn van buren 65725 20.0 +quinn young 65647 20.0 +quinn zipper 65579 20.0 +quinn zipper 65693 20.0 +rachel allen 65661 20.0 +rachel allen 65709 20.0 +rachel brown 65586 20.0 +rachel brown 65587 20.0 +rachel brown 65587 20.0 +rachel brown 65610 20.0 +rachel brown 65693 20.0 +rachel carson 65677 20.0 +rachel carson 65682 20.0 +rachel davidson 65755 20.0 +rachel ellison 65761 20.0 +rachel falkner 65616 20.0 +rachel falkner 65681 20.0 +rachel falkner 65693 20.0 +rachel falkner 65764 20.0 +rachel johnson 65658 20.0 +rachel king 65604 20.0 +rachel king 65643 20.0 +rachel laertes 65562 20.0 +rachel laertes 65624 20.0 +rachel ovid 65721 20.0 +rachel ovid 65736 20.0 +rachel polk 65686 20.0 +rachel quirinius 65787 20.0 +rachel robinson 65544 20.0 +rachel robinson 65717 20.0 +rachel robinson 65724 20.0 +rachel thompson 65648 20.0 +rachel thompson 65662 20.0 +rachel thompson 65733 20.0 +rachel underhill 65667 20.0 +rachel white 65615 20.0 +rachel white 65717 20.0 +rachel young 65727 20.0 +rachel zipper 65757 20.0 +rachel zipper 65785 20.0 +sarah carson 65679 20.0 +sarah carson 65693 20.0 +sarah carson 65694 20.0 +sarah ellison 65611 20.0 +sarah falkner 65606 20.0 +sarah falkner 65680 20.0 +sarah garcia 65563 20.0 +sarah garcia 65638 20.0 +sarah garcia 65661 20.0 +sarah ichabod 65667 20.0 +sarah ichabod 65671 20.0 +sarah johnson 65659 20.0 +sarah johnson 65716 20.0 +sarah johnson 65731 20.0 +sarah johnson 65751 20.0 +sarah king 65650 20.0 +sarah king 65699 20.0 +sarah miller 65557 20.0 +sarah ovid 65550 20.0 +sarah robinson 65677 20.0 +sarah robinson 65763 20.0 +sarah steinbeck 65721 20.0 +sarah white 65622 20.0 +sarah white 65747 20.0 +sarah xylophone 65678 20.0 +sarah young 65595 20.0 +sarah zipper 65550 20.0 +tom brown 65593 20.0 +tom brown 65675 20.0 +tom carson 65539 20.0 +tom carson 65624 20.0 +tom carson 65780 20.0 +tom davidson 65780 20.0 +tom ellison 65578 20.0 +tom ellison 65670 20.0 +tom ellison 65756 20.0 +tom falkner 65574 20.0 +tom falkner 65583 20.0 +tom hernandez 65575 20.0 +tom hernandez 65632 20.0 +tom ichabod 65588 20.0 +tom johnson 65536 20.0 +tom johnson 65789 20.0 +tom king 65576 20.0 +tom laertes 65617 20.0 +tom laertes 65701 20.0 +tom miller 65594 20.0 +tom miller 65603 20.0 +tom miller 65704 20.0 +tom nixon 65672 20.0 +tom ovid 65628 20.0 +tom polk 65652 20.0 +tom polk 65742 20.0 +tom quirinius 65563 20.0 +tom quirinius 65783 20.0 +tom robinson 65626 20.0 +tom robinson 65632 20.0 +tom robinson 65691 20.0 +tom robinson 65758 20.0 +tom steinbeck 65666 20.0 +tom van buren 65621 20.0 +tom van buren 65652 20.0 +tom van buren 65669 20.0 +tom white 65548 20.0 +tom young 65544 20.0 +tom young 65546 20.0 +tom zipper 65789 20.0 +ulysses brown 65735 20.0 +ulysses carson 65602 20.0 +ulysses carson 65643 20.0 +ulysses carson 65703 20.0 +ulysses carson 65716 20.0 +ulysses davidson 65750 20.0 +ulysses ellison 65575 20.0 +ulysses garcia 65666 20.0 +ulysses hernandez 65651 20.0 +ulysses hernandez 65702 20.0 +ulysses hernandez 65786 20.0 +ulysses ichabod 65551 20.0 +ulysses ichabod 65566 20.0 +ulysses johnson 65776 20.0 +ulysses king 65649 20.0 +ulysses laertes 65691 20.0 +ulysses laertes 65711 20.0 +ulysses laertes 65781 20.0 +ulysses miller 65610 20.0 +ulysses miller 65637 20.0 +ulysses nixon 65603 20.0 +ulysses ovid 65656 20.0 +ulysses polk 65563 20.0 +ulysses polk 65580 20.0 +ulysses polk 65612 20.0 +ulysses polk 65777 20.0 +ulysses quirinius 65786 20.0 +ulysses robinson 65744 20.0 +ulysses steinbeck 65611 20.0 +ulysses steinbeck 65680 20.0 +ulysses thompson 65788 20.0 +ulysses underhill 65570 20.0 +ulysses underhill 65616 20.0 +ulysses underhill 65620 20.0 +ulysses underhill 65623 20.0 +ulysses underhill 65641 20.0 +ulysses underhill 65713 20.0 +ulysses underhill 65785 20.0 +ulysses van buren 65684 20.0 +ulysses white 65654 20.0 +ulysses white 65675 20.0 +ulysses xylophone 65623 20.0 +ulysses xylophone 65636 20.0 +ulysses xylophone 65781 20.0 +ulysses young 65675 20.0 +ulysses young 65736 20.0 +ulysses young 65748 20.0 +victor allen 65684 20.0 +victor allen 65707 20.0 +victor brown 65550 20.0 +victor brown 65555 20.0 +victor brown 65622 20.0 +victor brown 65673 20.0 +victor davidson 65579 20.0 +victor davidson 65628 20.0 +victor davidson 65783 20.0 +victor ellison 65641 20.0 +victor ellison 65782 20.0 +victor hernandez 65571 20.0 +victor hernandez 65659 20.0 +victor hernandez 65708 20.0 +victor hernandez 65735 20.0 +victor hernandez 65775 20.0 +victor johnson 65606 20.0 +victor johnson 65607 20.0 +victor johnson 65607 20.0 +victor king 65721 20.0 +victor king 65743 20.0 +victor laertes 65638 20.0 +victor laertes 65644 20.0 +victor miller 65570 20.0 +victor nixon 65709 20.0 +victor nixon 65791 20.0 +victor ovid 65649 20.0 +victor polk 65625 20.0 +victor quirinius 65620 20.0 +victor quirinius 65651 20.0 +victor robinson 65596 20.0 +victor robinson 65673 20.0 +victor steinbeck 65618 20.0 +victor steinbeck 65661 20.0 +victor steinbeck 65686 20.0 +victor thompson 65548 20.0 +victor van buren 65664 20.0 +victor van buren 65774 20.0 +victor white 65548 20.0 +victor white 65601 20.0 +victor xylophone 65549 20.0 +victor xylophone 65618 20.0 +victor xylophone 65644 20.0 +victor xylophone 65677 20.0 +victor xylophone 65755 20.0 +victor young 65628 20.0 +victor zipper 65743 20.0 +wendy allen 65628 20.0 +wendy allen 65711 20.0 +wendy allen 65782 20.0 +wendy brown 65580 20.0 +wendy brown 65657 20.0 +wendy ellison 65545 20.0 +wendy ellison 65603 20.0 +wendy falkner 65595 20.0 +wendy falkner 65604 20.0 +wendy falkner 65635 20.0 +wendy garcia 65659 20.0 +wendy garcia 65746 20.0 +wendy garcia 65747 20.0 +wendy garcia 65777 20.0 +wendy hernandez 65650 20.0 +wendy ichabod 65730 20.0 +wendy king 65586 20.0 +wendy king 65664 20.0 +wendy king 65670 20.0 +wendy laertes 65566 20.0 +wendy laertes 65683 20.0 +wendy laertes 65727 20.0 +wendy miller 65582 20.0 +wendy miller 65626 20.0 +wendy nixon 65611 20.0 +wendy nixon 65746 20.0 +wendy ovid 65589 20.0 +wendy ovid 65643 20.0 +wendy polk 65656 20.0 +wendy polk 65692 20.0 +wendy quirinius 65766 20.0 +wendy quirinius 65767 20.0 +wendy robinson 65622 20.0 +wendy robinson 65715 20.0 +wendy robinson 65774 20.0 +wendy steinbeck 65612 20.0 +wendy thompson 65650 20.0 +wendy thompson 65737 20.0 +wendy underhill 65662 20.0 +wendy underhill 65758 20.0 +wendy underhill 65775 20.0 +wendy van buren 65680 20.0 +wendy van buren 65699 20.0 +wendy white 65705 20.0 +wendy xylophone 65687 20.0 +wendy xylophone 65773 20.0 +wendy young 65674 20.0 +wendy young 65685 20.0 +xavier allen 65611 20.0 +xavier allen 65618 20.0 +xavier allen 65771 20.0 +xavier brown 65600 20.0 +xavier brown 65704 20.0 +xavier brown 65723 20.0 +xavier carson 65731 20.0 +xavier carson 65758 20.0 +xavier davidson 65644 20.0 +xavier davidson 65664 20.0 +xavier davidson 65755 20.0 +xavier ellison 65541 20.0 +xavier ellison 65654 20.0 +xavier garcia 65672 20.0 +xavier hernandez 65541 20.0 +xavier hernandez 65544 20.0 +xavier hernandez 65766 20.0 +xavier ichabod 65597 20.0 +xavier ichabod 65663 20.0 +xavier johnson 65654 20.0 +xavier johnson 65744 20.0 +xavier king 65590 20.0 +xavier king 65601 20.0 +xavier laertes 65743 20.0 +xavier ovid 65788 20.0 +xavier polk 65587 20.0 +xavier polk 65653 20.0 +xavier polk 65675 20.0 +xavier polk 65696 20.0 +xavier quirinius 65599 20.0 +xavier quirinius 65650 20.0 +xavier quirinius 65656 20.0 +xavier quirinius 65737 20.0 +xavier thompson 65608 20.0 +xavier underhill 65710 20.0 +xavier white 65703 20.0 +xavier white 65732 20.0 +xavier xylophone 65572 20.0 +xavier zipper 65561 20.0 +yuri allen 65565 20.0 +yuri allen 65682 20.0 +yuri brown 65538 20.0 +yuri brown 65688 20.0 +yuri carson 65670 20.0 +yuri carson 65769 20.0 +yuri ellison 65570 20.0 +yuri ellison 65581 20.0 +yuri falkner 65658 20.0 +yuri falkner 65681 20.0 +yuri garcia 65639 20.0 +yuri hernandez 65706 20.0 +yuri johnson 65587 20.0 +yuri johnson 65697 20.0 +yuri johnson 65712 20.0 +yuri king 65721 20.0 +yuri laertes 65637 20.0 +yuri laertes 65773 20.0 +yuri nixon 65635 20.0 +yuri nixon 65740 20.0 +yuri polk 65607 20.0 +yuri polk 65713 20.0 +yuri polk 65742 20.0 +yuri quirinius 65544 20.0 +yuri quirinius 65617 20.0 +yuri quirinius 65695 20.0 +yuri steinbeck 65592 20.0 +yuri steinbeck 65679 20.0 +yuri thompson 65676 20.0 +yuri underhill 65718 20.0 +yuri underhill 65750 20.0 +yuri white 65659 20.0 +yuri xylophone 65714 20.0 +zach allen 65667 20.0 +zach brown 65559 20.0 +zach brown 65588 20.0 +zach brown 65691 20.0 +zach brown 65759 20.0 +zach brown 65762 20.0 +zach carson 65572 20.0 +zach ellison 65748 20.0 +zach falkner 65620 20.0 +zach falkner 65627 20.0 +zach garcia 65544 20.0 +zach garcia 65623 20.0 +zach garcia 65629 20.0 +zach garcia 65786 20.0 +zach ichabod 65599 20.0 +zach ichabod 65612 20.0 +zach king 65556 20.0 +zach king 65702 20.0 +zach king 65773 20.0 +zach miller 65584 20.0 +zach miller 65665 20.0 +zach miller 65719 20.0 +zach ovid 65578 20.0 +zach ovid 65703 20.0 +zach ovid 65750 20.0 +zach ovid 65784 20.0 +zach quirinius 65691 20.0 +zach robinson 65599 20.0 +zach steinbeck 65602 20.0 +zach steinbeck 65695 20.0 +zach thompson 65636 20.0 +zach thompson 65696 20.0 +zach underhill 65573 20.0 +zach white 65733 20.0 +zach xylophone 65542 20.0 +zach xylophone 65780 20.0 +zach young 65576 20.0 +zach zipper 65579 20.0 +zach zipper 65649 20.0 +zach zipper 65676 20.0 diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala index 027056d4b865f..5ead5f3c19908 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala @@ -300,7 +300,7 @@ abstract class HiveComparisonTest val hiveQueries = queryList.map(new TestHive.HiveQLQueryExecution(_)) // Make sure we can at least parse everything before attempting hive execution. - hiveQueries.foreach(_.analyzed) + hiveQueries.foreach(_.logical) val computedResults = (queryList.zipWithIndex, hiveQueries, hiveCacheFiles).zipped.map { case ((queryString, i), hiveQuery, cachedAnswerFile)=> try { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala new file mode 100644 index 0000000000000..934452fe579a1 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala @@ -0,0 +1,845 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.execution + +import java.io.File +import java.util.{Locale, TimeZone} + +import org.scalatest.BeforeAndAfter + +import org.apache.spark.sql.hive.test.TestHive +import org.apache.spark.sql.hive.test.TestHive._ +import org.apache.spark.util.Utils + +/** + * The test suite for window functions. To actually compare results with Hive, + * every test should be created by `createQueryTest`. Because we are reusing tables + * for different tests and there are a few properties needed to let Hive generate golden + * files, every `createQueryTest` calls should explicitly set `reset` to `false`. + */ +abstract class HiveWindowFunctionQueryBaseSuite extends HiveComparisonTest with BeforeAndAfter { + private val originalTimeZone = TimeZone.getDefault + private val originalLocale = Locale.getDefault + private val testTempDir = Utils.createTempDir() + + override def beforeAll() { + TestHive.cacheTables = true + // Timezone is fixed to America/Los_Angeles for those timezone sensitive tests (timestamp_*) + TimeZone.setDefault(TimeZone.getTimeZone("America/Los_Angeles")) + // Add Locale setting + Locale.setDefault(Locale.US) + + // Create the table used in windowing.q + sql("DROP TABLE IF EXISTS part") + sql( + """ + |CREATE TABLE part( + | p_partkey INT, + | p_name STRING, + | p_mfgr STRING, + | p_brand STRING, + | p_type STRING, + | p_size INT, + | p_container STRING, + | p_retailprice DOUBLE, + | p_comment STRING) + """.stripMargin) + val testData1 = TestHive.getHiveFile("data/files/part_tiny.txt").getCanonicalPath + sql( + s""" + |LOAD DATA LOCAL INPATH '$testData1' overwrite into table part + """.stripMargin) + + sql("DROP TABLE IF EXISTS over1k") + sql( + """ + |create table over1k( + | t tinyint, + | si smallint, + | i int, + | b bigint, + | f float, + | d double, + | bo boolean, + | s string, + | ts timestamp, + | dec decimal(4,2), + | bin binary) + |row format delimited + |fields terminated by '|' + """.stripMargin) + val testData2 = TestHive.getHiveFile("data/files/over1k").getCanonicalPath + sql( + s""" + |LOAD DATA LOCAL INPATH '$testData2' overwrite into table over1k + """.stripMargin) + + // The following settings are used for generating golden files with Hive. + // We have to use kryo to correctly let Hive serialize plans with window functions. + // This is used to generate golden files. + sql("set hive.plan.serialization.format=kryo") + // Explicitly set fs to local fs. + sql(s"set fs.default.name=file://$testTempDir/") + // Ask Hive to run jobs in-process as a single map and reduce task. + sql("set mapred.job.tracker=local") + } + + override def afterAll() { + TestHive.cacheTables = false + TimeZone.setDefault(originalTimeZone) + Locale.setDefault(originalLocale) + TestHive.reset() + } + + ///////////////////////////////////////////////////////////////////////////// + // Tests based on windowing_multipartitioning.q + // Results of the original query file are not deterministic. + ///////////////////////////////////////////////////////////////////////////// + createQueryTest("windowing_multipartitioning.q (deterministic) 1", + s""" + |select s, + |rank() over (partition by s order by si) r, + |sum(b) over (partition by s order by si) sum + |from over1k + |order by s, r, sum; + """.stripMargin, reset = false) + + /* timestamp comparison issue with Hive? + createQueryTest("windowing_multipartitioning.q (deterministic) 2", + s""" + |select s, + |rank() over (partition by s order by dec desc) r, + |sum(b) over (partition by s order by ts desc) as sum + |from over1k + |where s = 'tom allen' or s = 'bob steinbeck' + |order by s, r, sum; + """.stripMargin, reset = false) + */ + + createQueryTest("windowing_multipartitioning.q (deterministic) 3", + s""" + |select s, sum(i) over (partition by s), sum(f) over (partition by si) + |from over1k where s = 'tom allen' or s = 'bob steinbeck'; + """.stripMargin, reset = false) + + createQueryTest("windowing_multipartitioning.q (deterministic) 4", + s""" + |select s, rank() over (partition by s order by bo), + |rank() over (partition by si order by bin desc) from over1k + |where s = 'tom allen' or s = 'bob steinbeck'; + """.stripMargin, reset = false) + + createQueryTest("windowing_multipartitioning.q (deterministic) 5", + s""" + |select s, sum(f) over (partition by i), row_number() over (order by f) + |from over1k where s = 'tom allen' or s = 'bob steinbeck'; + """.stripMargin, reset = false) + + createQueryTest("windowing_multipartitioning.q (deterministic) 6", + s""" + |select s, rank() over w1, + |rank() over w2 + |from over1k + |where s = 'tom allen' or s = 'bob steinbeck' + |window + |w1 as (partition by s order by dec), + |w2 as (partition by si order by f) ; + """.stripMargin, reset = false) + + ///////////////////////////////////////////////////////////////////////////// + // Tests based on windowing_navfn.q + // Results of the original query file are not deterministic. + // Also, the original query of + // select i, lead(s) over (partition by bin order by d,i desc) from over1k ; + ///////////////////////////////////////////////////////////////////////////// + createQueryTest("windowing_navfn.q (deterministic)", + s""" + |select s, row_number() over (partition by d order by dec) rn from over1k + |order by s, rn desc; + |select i, lead(s) over (partition by cast(bin as string) order by d,i desc) as l + |from over1k + |order by i desc, l; + |select i, lag(dec) over (partition by i order by s,i,dec) l from over1k + |order by i, l; + |select s, last_value(t) over (partition by d order by f) l from over1k + |order by s, l; + |select s, first_value(s) over (partition by bo order by s) f from over1k + |order by s, f; + |select t, s, i, last_value(i) over (partition by t order by s) + |from over1k where (s = 'oscar allen' or s = 'oscar carson') and t = 10; + """.stripMargin, reset = false) + + ///////////////////////////////////////////////////////////////////////////// + // Tests based on windowing_ntile.q + // Results of the original query file are not deterministic. + ///////////////////////////////////////////////////////////////////////////// + createQueryTest("windowing_ntile.q (deterministic)", + s""" + |select i, ntile(10) over (partition by s order by i) n from over1k + |order by i, n; + |select s, ntile(100) over (partition by i order by s) n from over1k + |order by s, n; + |select f, ntile(4) over (partition by d order by f) n from over1k + |order by f, n; + |select d, ntile(1000) over (partition by dec order by d) n from over1k + |order by d, n; + """.stripMargin, reset = false) + + ///////////////////////////////////////////////////////////////////////////// + // Tests based on windowing_udaf.q + // Results of the original query file are not deterministic. + ///////////////////////////////////////////////////////////////////////////// + createQueryTest("windowing_udaf.q (deterministic)", + s""" + |select s, min(i) over (partition by s) m from over1k + |order by s, m; + |select s, avg(f) over (partition by si order by s) a from over1k + |order by s, a; + |select s, avg(i) over (partition by t, b order by s) a from over1k + |order by s, a; + |select max(i) over w m from over1k + |order by m window w as (partition by f) ; + |select s, avg(d) over (partition by t order by f) a from over1k + |order by s, a; + """.stripMargin, reset = false) + + ///////////////////////////////////////////////////////////////////////////// + // Tests based on windowing_windowspec.q + // Results of the original query file are not deterministic. + ///////////////////////////////////////////////////////////////////////////// + createQueryTest("windowing_windowspec.q (deterministic)", + s""" + |select s, sum(b) over (partition by i order by s,b rows unbounded preceding) as sum + |from over1k order by s, sum; + |select s, sum(f) over (partition by d order by s,f rows unbounded preceding) as sum + |from over1k order by s, sum; + |select s, sum(f) over + |(partition by ts order by f range between current row and unbounded following) as sum + |from over1k order by s, sum; + |select s, avg(f) + |over (partition by ts order by s,f rows between current row and 5 following) avg + |from over1k order by s, avg; + |select s, avg(d) over + |(partition by t order by s,d desc rows between 5 preceding and 5 following) avg + |from over1k order by s, avg; + |select s, sum(i) over(partition by ts order by s) sum from over1k + |order by s, sum; + |select f, sum(f) over + |(partition by ts order by f range between unbounded preceding and current row) sum + |from over1k order by f, sum; + |select s, i, round(avg(d) over (partition by s order by i) / 10.0 , 2) avg + |from over1k order by s, i, avg; + |select s, i, round((avg(d) over w1 + 10.0) - (avg(d) over w1 - 10.0),2) avg + |from over1k + |order by s, i, avg window w1 as (partition by s order by i); + """.stripMargin, reset = false) + + ///////////////////////////////////////////////////////////////////////////// + // Tests based on windowing_rank.q + // Results of the original query file are not deterministic. + ///////////////////////////////////////////////////////////////////////////// + createQueryTest("windowing_rank.q (deterministic) 1", + s""" + |select s, rank() over (partition by f order by t) r from over1k order by s, r; + |select s, dense_rank() over (partition by ts order by i,s desc) as r from over1k + |order by s desc, r desc; + |select s, cume_dist() over (partition by bo order by b,s) cd from over1k + |order by s, cd; + |select s, percent_rank() over (partition by dec order by f) r from over1k + |order by s desc, r desc; + """.stripMargin, reset = false) + + createQueryTest("windowing_rank.q (deterministic) 2", + s""" + |select ts, dec, rnk + |from + | (select ts, dec, + | rank() over (partition by ts order by dec) as rnk + | from + | (select other.ts, other.dec + | from over1k other + | join over1k on (other.b = over1k.b) + | ) joined + | ) ranked + |where rnk = 1 + |order by ts, dec, rnk; + """.stripMargin, reset = false) + + createQueryTest("windowing_rank.q (deterministic) 3", + s""" + |select ts, dec, rnk + |from + | (select ts, dec, + | rank() over (partition by ts order by dec) as rnk + | from + | (select other.ts, other.dec + | from over1k other + | join over1k on (other.b = over1k.b) + | ) joined + | ) ranked + |where dec = 89.5 + |order by ts, dec, rnk; + """.stripMargin, reset = false) + + createQueryTest("windowing_rank.q (deterministic) 4", + s""" + |select ts, dec, rnk + |from + | (select ts, dec, + | rank() over (partition by ts order by dec) as rnk + | from + | (select other.ts, other.dec + | from over1k other + | join over1k on (other.b = over1k.b) + | where other.t < 10 + | ) joined + | ) ranked + |where rnk = 1 + |order by ts, dec, rnk; + """.stripMargin, reset = false) + + ///////////////////////////////////////////////////////////////////////////// + // Tests from windowing.q + // We port tests in windowing.q to here because this query file contains too + // many tests and the syntax of test "-- 7. testJoinWithWindowingAndPTF" + // is not supported right now. + ///////////////////////////////////////////////////////////////////////////// + createQueryTest("windowing.q -- 1. testWindowing", + s""" + |select p_mfgr, p_name, p_size, + |rank() over(distribute by p_mfgr sort by p_name) as r, + |dense_rank() over(distribute by p_mfgr sort by p_name) as dr, + |sum(p_retailprice) over + |(distribute by p_mfgr sort by p_name rows between unbounded preceding and current row) as s1 + |from part + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 2. testGroupByWithPartitioning", + s""" + |select p_mfgr, p_name, p_size, + |min(p_retailprice), + |rank() over(distribute by p_mfgr sort by p_name)as r, + |dense_rank() over(distribute by p_mfgr sort by p_name) as dr, + |p_size, p_size - lag(p_size,1,p_size) over(distribute by p_mfgr sort by p_name) as deltaSz + |from part + |group by p_mfgr, p_name, p_size + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 3. testGroupByHavingWithSWQ", + s""" + |select p_mfgr, p_name, p_size, min(p_retailprice), + |rank() over(distribute by p_mfgr sort by p_name) as r, + |dense_rank() over(distribute by p_mfgr sort by p_name) as dr, + |p_size, p_size - lag(p_size,1,p_size) over(distribute by p_mfgr sort by p_name) as deltaSz + |from part + |group by p_mfgr, p_name, p_size + |having p_size > 0 + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 4. testCount", + s""" + |select p_mfgr, p_name, + |count(p_size) over(distribute by p_mfgr sort by p_name) as cd + |from part + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 5. testCountWithWindowingUDAF", + s""" + |select p_mfgr, p_name, + |rank() over(distribute by p_mfgr sort by p_name) as r, + |dense_rank() over(distribute by p_mfgr sort by p_name) as dr, + |count(p_size) over(distribute by p_mfgr sort by p_name) as cd, + |p_retailprice, sum(p_retailprice) over (distribute by p_mfgr sort by p_name + | rows between unbounded preceding and current row) as s1, + |p_size, p_size - lag(p_size,1,p_size) over(distribute by p_mfgr sort by p_name) as deltaSz + |from part + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 6. testCountInSubQ", + s""" + |select sub1.r, sub1.dr, sub1.cd, sub1.s1, sub1.deltaSz + |from (select p_mfgr, p_name, + |rank() over(distribute by p_mfgr sort by p_name) as r, + |dense_rank() over(distribute by p_mfgr sort by p_name) as dr, + |count(p_size) over(distribute by p_mfgr sort by p_name) as cd, + |p_retailprice, sum(p_retailprice) over (distribute by p_mfgr sort by p_name + | rows between unbounded preceding and current row) as s1, + |p_size, p_size - lag(p_size,1,p_size) over(distribute by p_mfgr sort by p_name) as deltaSz + |from part + |) sub1 + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 8. testMixedCaseAlias", + s""" + |select p_mfgr, p_name, p_size, + |rank() over(distribute by p_mfgr sort by p_name, p_size desc) as R + |from part + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 9. testHavingWithWindowingNoGBY", + s""" + |select p_mfgr, p_name, p_size, + |rank() over(distribute by p_mfgr sort by p_name) as r, + |dense_rank() over(distribute by p_mfgr sort by p_name) as dr, + |sum(p_retailprice) over (distribute by p_mfgr sort by p_name + | rows between unbounded preceding and current row) as s1 + |from part + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 10. testHavingWithWindowingCondRankNoGBY", + s""" + |select p_mfgr, p_name, p_size, + |rank() over(distribute by p_mfgr sort by p_name) as r, + |dense_rank() over(distribute by p_mfgr sort by p_name) as dr, + |sum(p_retailprice) over (distribute by p_mfgr sort by p_name + | rows between unbounded preceding and current row) as s1 + |from part + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 11. testFirstLast", + s""" + |select p_mfgr,p_name, p_size, + |sum(p_size) over (distribute by p_mfgr sort by p_name + |rows between current row and current row) as s2, + |first_value(p_size) over w1 as f, + |last_value(p_size, false) over w1 as l + |from part + |window w1 as (distribute by p_mfgr sort by p_name rows between 2 preceding and 2 following) + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 12. testFirstLastWithWhere", + s""" + |select p_mfgr,p_name, p_size, + |rank() over(distribute by p_mfgr sort by p_name) as r, + |sum(p_size) over (distribute by p_mfgr sort by p_name + |rows between current row and current row) as s2, + |first_value(p_size) over w1 as f, + |last_value(p_size, false) over w1 as l + |from part + |where p_mfgr = 'Manufacturer#3' + |window w1 as (distribute by p_mfgr sort by p_name rows between 2 preceding and 2 following) + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 13. testSumWindow", + s""" + |select p_mfgr,p_name, p_size, + |sum(p_size) over w1 as s1, + |sum(p_size) over (distribute by p_mfgr sort by p_name + |rows between current row and current row) as s2 + |from part + |window w1 as (distribute by p_mfgr sort by p_name rows between 2 preceding and 2 following) + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 14. testNoSortClause", + s""" + |select p_mfgr,p_name, p_size, + |rank() over(distribute by p_mfgr sort by p_name) as r, + |dense_rank() over(distribute by p_mfgr sort by p_name) as dr + |from part + |window w1 as (distribute by p_mfgr sort by p_name rows between 2 preceding and 2 following) + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 15. testExpressions", + s""" + |select p_mfgr,p_name, p_size, + |rank() over(distribute by p_mfgr sort by p_name) as r, + |dense_rank() over(distribute by p_mfgr sort by p_name) as dr, + |cume_dist() over(distribute by p_mfgr sort by p_name) as cud, + |percent_rank() over(distribute by p_mfgr sort by p_name) as pr, + |ntile(3) over(distribute by p_mfgr sort by p_name) as nt, + |count(p_size) over(distribute by p_mfgr sort by p_name) as ca, + |avg(p_size) over(distribute by p_mfgr sort by p_name) as avg, + |stddev(p_size) over(distribute by p_mfgr sort by p_name) as st, + |first_value(p_size % 5) over(distribute by p_mfgr sort by p_name) as fv, + |last_value(p_size) over(distribute by p_mfgr sort by p_name) as lv, + |first_value(p_size) over w1 as fvW1 + |from part + |window w1 as (distribute by p_mfgr sort by p_mfgr, p_name + | rows between 2 preceding and 2 following) + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 16. testMultipleWindows", + s""" + |select p_mfgr,p_name, p_size, + |rank() over(distribute by p_mfgr sort by p_name) as r, + |dense_rank() over(distribute by p_mfgr sort by p_name) as dr, + |cume_dist() over(distribute by p_mfgr sort by p_name) as cud, + |sum(p_size) over (distribute by p_mfgr sort by p_name + |range between unbounded preceding and current row) as s1, + |sum(p_size) over (distribute by p_mfgr sort by p_size + |range between 5 preceding and current row) as s2, + |first_value(p_size) over w1 as fv1 + |from part + |window w1 as (distribute by p_mfgr sort by p_mfgr, p_name + | rows between 2 preceding and 2 following) + """.stripMargin, reset = false) + + + createQueryTest("windowing.q -- 17. testCountStar", + s""" + |select p_mfgr,p_name, p_size, + |count(*) over(distribute by p_mfgr sort by p_name ) as c, + |count(p_size) over(distribute by p_mfgr sort by p_name) as ca, + |first_value(p_size) over w1 as fvW1 + |from part + |window w1 as (distribute by p_mfgr sort by p_mfgr, p_name + | rows between 2 preceding and 2 following) + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 18. testUDAFs", + s""" + |select p_mfgr,p_name, p_size, + |sum(p_retailprice) over w1 as s, + |min(p_retailprice) over w1 as mi, + |max(p_retailprice) over w1 as ma, + |avg(p_retailprice) over w1 as ag + |from part + |window w1 as (distribute by p_mfgr sort by p_mfgr, p_name + | rows between 2 preceding and 2 following) + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 19. testUDAFsWithGBY", + """ + |select p_mfgr,p_name, p_size, p_retailprice, + |sum(p_retailprice) over w1 as s, + |min(p_retailprice) as mi , + |max(p_retailprice) as ma , + |avg(p_retailprice) over w1 as ag + |from part + |group by p_mfgr,p_name, p_size, p_retailprice + |window w1 as (distribute by p_mfgr sort by p_mfgr, p_name + | rows between 2 preceding and 2 following); + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 20. testSTATs", + """ + |select p_mfgr,p_name, p_size, + |stddev(p_retailprice) over w1 as sdev, + |stddev_pop(p_retailprice) over w1 as sdev_pop, + |collect_set(p_size) over w1 as uniq_size, + |variance(p_retailprice) over w1 as var, + |corr(p_size, p_retailprice) over w1 as cor, + |covar_pop(p_size, p_retailprice) over w1 as covarp + |from part + |window w1 as (distribute by p_mfgr sort by p_mfgr, p_name + | rows between 2 preceding and 2 following) + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 21. testDISTs", + """ + |select p_mfgr,p_name, p_size, + |histogram_numeric(p_retailprice, 5) over w1 as hist, + |percentile(p_partkey, 0.5) over w1 as per, + |row_number() over(distribute by p_mfgr sort by p_name) as rn + |from part + |window w1 as (distribute by p_mfgr sort by p_mfgr, p_name + | rows between 2 preceding and 2 following) + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 24. testLateralViews", + """ + |select p_mfgr, p_name, + |lv_col, p_size, sum(p_size) over w1 as s + |from (select p_mfgr, p_name, p_size, array(1,2,3) arr from part) p + |lateral view explode(arr) part_lv as lv_col + |window w1 as (distribute by p_mfgr sort by p_size, lv_col + | rows between 2 preceding and current row) + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 26. testGroupByHavingWithSWQAndAlias", + """ + |select p_mfgr, p_name, p_size, min(p_retailprice) as mi, + |rank() over(distribute by p_mfgr sort by p_name) as r, + |dense_rank() over(distribute by p_mfgr sort by p_name) as dr, + |p_size, p_size - lag(p_size,1,p_size) over(distribute by p_mfgr sort by p_name) as deltaSz + |from part + |group by p_mfgr, p_name, p_size + |having p_size > 0 + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 27. testMultipleRangeWindows", + """ + |select p_mfgr,p_name, p_size, + |sum(p_size) over (distribute by p_mfgr sort by p_size + |range between 10 preceding and current row) as s2, + |sum(p_size) over (distribute by p_mfgr sort by p_size + |range between current row and 10 following ) as s1 + |from part + |window w1 as (rows between 2 preceding and 2 following) + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 28. testPartOrderInUDAFInvoke", + """ + |select p_mfgr, p_name, p_size, + |sum(p_size) over (partition by p_mfgr order by p_name + |rows between 2 preceding and 2 following) as s + |from part + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 29. testPartOrderInWdwDef", + """ + |select p_mfgr, p_name, p_size, + |sum(p_size) over w1 as s + |from part + |window w1 as (partition by p_mfgr order by p_name + | rows between 2 preceding and 2 following) + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 30. testDefaultPartitioningSpecRules", + """ + |select p_mfgr, p_name, p_size, + |sum(p_size) over w1 as s, + |sum(p_size) over w2 as s2 + |from part + |window w1 as (distribute by p_mfgr sort by p_name rows between 2 preceding and 2 following), + | w2 as (partition by p_mfgr order by p_name) + """.stripMargin, reset = false) + + /* p_name is not a numeric column. What is Hive's semantic? + createQueryTest("windowing.q -- 31. testWindowCrossReference", + """ + |select p_mfgr, p_name, p_size, + |sum(p_size) over w1 as s1, + |sum(p_size) over w2 as s2 + |from part + |window w1 as (partition by p_mfgr order by p_name + | range between 2 preceding and 2 following), + | w2 as w1 + """.stripMargin, reset = false) + */ + /* + createQueryTest("windowing.q -- 32. testWindowInheritance", + """ + |select p_mfgr, p_name, p_size, + |sum(p_size) over w1 as s1, + |sum(p_size) over w2 as s2 + |from part + |window w1 as (partition by p_mfgr order by p_name + | range between 2 preceding and 2 following), + | w2 as (w1 rows between unbounded preceding and current row) + """.stripMargin, reset = false) + */ + + /* p_name is not a numeric column. What is Hive's semantic? + createQueryTest("windowing.q -- 33. testWindowForwardReference", + """ + |select p_mfgr, p_name, p_size, + |sum(p_size) over w1 as s1, + |sum(p_size) over w2 as s2, + |sum(p_size) over w3 as s3 + |from part + |window w1 as (distribute by p_mfgr sort by p_name + | range between 2 preceding and 2 following), + | w2 as w3, + | w3 as (distribute by p_mfgr sort by p_name + | range between unbounded preceding and current row) + """.stripMargin, reset = false) + */ + /* + createQueryTest("windowing.q -- 34. testWindowDefinitionPropagation", + """ + |select p_mfgr, p_name, p_size, + |sum(p_size) over w1 as s1, + |sum(p_size) over w2 as s2, + |sum(p_size) over (w3 rows between 2 preceding and 2 following) as s3 + |from part + |window w1 as (distribute by p_mfgr sort by p_name + | range between 2 preceding and 2 following), + | w2 as w3, + | w3 as (distribute by p_mfgr sort by p_name + | range between unbounded preceding and current row) + """.stripMargin, reset = false) + */ + + /* Seems Hive evaluate SELECT DISTINCT before window functions? + createQueryTest("windowing.q -- 35. testDistinctWithWindowing", + """ + |select DISTINCT p_mfgr, p_name, p_size, + |sum(p_size) over w1 as s + |from part + |window w1 as (distribute by p_mfgr sort by p_name rows between 2 preceding and 2 following) + """.stripMargin, reset = false) + */ + + createQueryTest("windowing.q -- 36. testRankWithPartitioning", + """ + |select p_mfgr, p_name, p_size, + |rank() over (partition by p_mfgr order by p_name ) as r + |from part + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 37. testPartitioningVariousForms", + """ + |select p_mfgr, + |round(sum(p_retailprice) over (partition by p_mfgr order by p_mfgr),2) as s1, + |min(p_retailprice) over (partition by p_mfgr) as s2, + |max(p_retailprice) over (distribute by p_mfgr sort by p_mfgr) as s3, + |round(avg(p_retailprice) over (distribute by p_mfgr),2) as s4, + |count(p_retailprice) over (cluster by p_mfgr ) as s5 + |from part + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 38. testPartitioningVariousForms2", + """ + |select p_mfgr, p_name, p_size, + |sum(p_retailprice) over (partition by p_mfgr, p_name order by p_mfgr, p_name + |rows between unbounded preceding and current row) as s1, + |min(p_retailprice) over (distribute by p_mfgr, p_name sort by p_mfgr, p_name + |rows between unbounded preceding and current row) as s2, + |max(p_retailprice) over (partition by p_mfgr, p_name order by p_name) as s3 + |from part + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 39. testUDFOnOrderCols", + """ + |select p_mfgr, p_type, substr(p_type, 2) as short_ptype, + |rank() over (partition by p_mfgr order by substr(p_type, 2)) as r + |from part + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 40. testNoBetweenForRows", + """ + |select p_mfgr, p_name, p_size, + |sum(p_retailprice) over (distribute by p_mfgr sort by p_name rows unbounded preceding) as s1 + |from part + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 41. testNoBetweenForRange", + """ + |select p_mfgr, p_name, p_size, + |sum(p_retailprice) over (distribute by p_mfgr sort by p_size range unbounded preceding) as s1 + |from part + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 42. testUnboundedFollowingForRows", + """ + |select p_mfgr, p_name, p_size, + |sum(p_retailprice) over (distribute by p_mfgr sort by p_name + |rows between current row and unbounded following) as s1 + |from part + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 43. testUnboundedFollowingForRange", + """ + |select p_mfgr, p_name, p_size, + |sum(p_retailprice) over (distribute by p_mfgr sort by p_size + |range between current row and unbounded following) as s1 + |from part + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 44. testOverNoPartitionSingleAggregate", + """ + |select p_name, p_retailprice, + |round(avg(p_retailprice) over(),2) + |from part + |order by p_name + """.stripMargin, reset = false) +} + +class HiveWindowFunctionQueryWithoutCodeGenSuite extends HiveWindowFunctionQueryBaseSuite { + var originalCodegenEnabled: Boolean = _ + override def beforeAll(): Unit = { + super.beforeAll() + originalCodegenEnabled = conf.codegenEnabled + sql("set spark.sql.codegen=false") + } + + override def afterAll(): Unit = { + sql(s"set spark.sql.codegen=$originalCodegenEnabled") + super.afterAll() + } +} + +abstract class HiveWindowFunctionQueryFileBaseSuite + extends HiveCompatibilitySuite with BeforeAndAfter { + private val originalTimeZone = TimeZone.getDefault + private val originalLocale = Locale.getDefault + private val testTempDir = Utils.createTempDir() + + override def beforeAll() { + TestHive.cacheTables = true + // Timezone is fixed to America/Los_Angeles for those timezone sensitive tests (timestamp_*) + TimeZone.setDefault(TimeZone.getTimeZone("America/Los_Angeles")) + // Add Locale setting + Locale.setDefault(Locale.US) + + // The following settings are used for generating golden files with Hive. + // We have to use kryo to correctly let Hive serialize plans with window functions. + // This is used to generate golden files. + sql("set hive.plan.serialization.format=kryo") + // Explicitly set fs to local fs. + sql(s"set fs.default.name=file://$testTempDir/") + // Ask Hive to run jobs in-process as a single map and reduce task. + sql("set mapred.job.tracker=local") + } + + override def afterAll() { + TestHive.cacheTables = false + TimeZone.setDefault(originalTimeZone) + Locale.setDefault(originalLocale) + TestHive.reset() + } + + override def blackList: Seq[String] = Seq( + // Partitioned table functions are not supported. + "ptf*", + // tests of windowing.q are in HiveWindowFunctionQueryBaseSuite + "windowing.q", + + // This one failed on the expression of + // sum(lag(p_retailprice,1,0.0)) over w1 + // lag(p_retailprice,1,0.0) is a GenericUDF and the argument inspector of + // p_retailprice created by HiveInspectors is + // PrimitiveObjectInspectorFactory.javaDoubleObjectInspector. + // However, seems Hive assumes it is + // PrimitiveObjectInspectorFactory.writableDoubleObjectInspector, which introduces an error. + "windowing_expressions", + + // Hive's results are not deterministic + "windowing_multipartitioning", + "windowing_navfn", + "windowing_ntile", + "windowing_udaf", + "windowing_windowspec", + "windowing_rank" + ) + + override def whiteList: Seq[String] = Seq( + "windowing_udaf2", + "windowing_columnPruning", + "windowing_adjust_rowcontainer_sz" + ) + + override def testCases: Seq[(String, File)] = super.testCases.filter { + case (name, _) => realWhiteList.contains(name) + } +} + +class HiveWindowFunctionQueryFileWithoutCodeGenSuite extends HiveWindowFunctionQueryFileBaseSuite { + var originalCodegenEnabled: Boolean = _ + override def beforeAll(): Unit = { + super.beforeAll() + originalCodegenEnabled = conf.codegenEnabled + sql("set spark.sql.codegen=false") + } + + override def afterAll(): Unit = { + sql(s"set spark.sql.codegen=$originalCodegenEnabled") + super.afterAll() + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index 630dec8fa05a0..616352d223c5b 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -48,6 +48,10 @@ case class Order( state: String, month: Int) +case class WindowData( + month: Int, + area: String, + product: Int) /** A SQL Dialect for testing purpose, and it can not be nested type */ class MyDialect extends DefaultDialect @@ -604,4 +608,147 @@ class SQLQuerySuite extends QueryTest { sql("SELECT TRANSFORM (d1, d2, d3) USING 'cat' AS (a,b,c) FROM script_trans") .queryExecution.toRdd.count()) } + + test("window function: udaf with aggregate expressin") { + val data = Seq( + WindowData(1, "a", 5), + WindowData(2, "a", 6), + WindowData(3, "b", 7), + WindowData(4, "b", 8), + WindowData(5, "c", 9), + WindowData(6, "c", 10) + ) + sparkContext.parallelize(data).toDF().registerTempTable("windowData") + + checkAnswer( + sql( + """ + |select area, sum(product), sum(sum(product)) over (partition by area) + |from windowData group by month, area + """.stripMargin), + Seq( + ("a", 5, 11), + ("a", 6, 11), + ("b", 7, 15), + ("b", 8, 15), + ("c", 9, 19), + ("c", 10, 19) + ).map(i => Row(i._1, i._2, i._3))) + + checkAnswer( + sql( + """ + |select area, sum(product) - 1, sum(sum(product)) over (partition by area) + |from windowData group by month, area + """.stripMargin), + Seq( + ("a", 4, 11), + ("a", 5, 11), + ("b", 6, 15), + ("b", 7, 15), + ("c", 8, 19), + ("c", 9, 19) + ).map(i => Row(i._1, i._2, i._3))) + + checkAnswer( + sql( + """ + |select area, sum(product), sum(product) / sum(sum(product)) over (partition by area) + |from windowData group by month, area + """.stripMargin), + Seq( + ("a", 5, 5d/11), + ("a", 6, 6d/11), + ("b", 7, 7d/15), + ("b", 8, 8d/15), + ("c", 10, 10d/19), + ("c", 9, 9d/19) + ).map(i => Row(i._1, i._2, i._3))) + + checkAnswer( + sql( + """ + |select area, sum(product), sum(product) / sum(sum(product) - 1) over (partition by area) + |from windowData group by month, area + """.stripMargin), + Seq( + ("a", 5, 5d/9), + ("a", 6, 6d/9), + ("b", 7, 7d/13), + ("b", 8, 8d/13), + ("c", 10, 10d/17), + ("c", 9, 9d/17) + ).map(i => Row(i._1, i._2, i._3))) + } + + test("window function: partition and order expressions") { + val data = Seq( + WindowData(1, "a", 5), + WindowData(2, "a", 6), + WindowData(3, "b", 7), + WindowData(4, "b", 8), + WindowData(5, "c", 9), + WindowData(6, "c", 10) + ) + sparkContext.parallelize(data).toDF().registerTempTable("windowData") + + checkAnswer( + sql( + """ + |select month, area, product, sum(product + 1) over (partition by 1 order by 2) + |from windowData + """.stripMargin), + Seq( + (1, "a", 5, 51), + (2, "a", 6, 51), + (3, "b", 7, 51), + (4, "b", 8, 51), + (5, "c", 9, 51), + (6, "c", 10, 51) + ).map(i => Row(i._1, i._2, i._3, i._4))) + + checkAnswer( + sql( + """ + |select month, area, product, sum(product) + |over (partition by month % 2 order by 10 - product) + |from windowData + """.stripMargin), + Seq( + (1, "a", 5, 21), + (2, "a", 6, 24), + (3, "b", 7, 16), + (4, "b", 8, 18), + (5, "c", 9, 9), + (6, "c", 10, 10) + ).map(i => Row(i._1, i._2, i._3, i._4))) + } + + test("window function: expressions in arguments of a window functions") { + val data = Seq( + WindowData(1, "a", 5), + WindowData(2, "a", 6), + WindowData(3, "b", 7), + WindowData(4, "b", 8), + WindowData(5, "c", 9), + WindowData(6, "c", 10) + ) + sparkContext.parallelize(data).toDF().registerTempTable("windowData") + + checkAnswer( + sql( + """ + |select month, area, month % 2, + |lag(product, 1 + 1, product) over (partition by month % 2 order by area) + |from windowData + """.stripMargin), + Seq( + (1, "a", 1, 5), + (2, "a", 0, 6), + (3, "b", 1, 7), + (4, "b", 0, 8), + (5, "c", 1, 5), + (6, "c", 0, 6) + ).map(i => Row(i._1, i._2, i._3, i._4))) + } } From 002c12384d6ecebbb3e7fc853dbdfbc5aaa3d6a6 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Wed, 6 May 2015 10:52:55 -0700 Subject: [PATCH 13/75] [SPARK-7311] Introduce internal Serializer API for determining if serializers support object relocation This patch extends the `Serializer` interface with a new `Private` API which allows serializers to indicate whether they support relocation of serialized objects in serializer stream output. This relocatibilty property is described in more detail in `Serializer.scala`, but in a nutshell a serializer supports relocation if reordering the bytes of serialized objects in serialization stream output is equivalent to having re-ordered those elements prior to serializing them. The optimized shuffle path introduced in #4450 and #5868 both rely on serializers having this property; this patch just centralizes the logic for determining whether a serializer has this property. I also added tests and comments clarifying when this works for KryoSerializer. This change allows the optimizations in #4450 to be applied for shuffles that use `SqlSerializer2`. Author: Josh Rosen Closes #5924 from JoshRosen/SPARK-7311 and squashes the following commits: 50a68ca [Josh Rosen] Address minor nits 0a7ebd7 [Josh Rosen] Clarify reason why SqlSerializer2 supports this serializer 123b992 [Josh Rosen] Cleanup for submitting as standalone patch. 4aa61b2 [Josh Rosen] Add missing newline 2c1233a [Josh Rosen] Small refactoring of SerializerPropertiesSuite to enable test re-use: 0ba75e6 [Josh Rosen] Add tests for serializer relocation property. 450fa21 [Josh Rosen] Back out accidental log4j.properties change 86d4dcd [Josh Rosen] Flag that SparkSqlSerializer2 supports relocation b9624ee [Josh Rosen] Expand serializer API and use new function to help control when new UnsafeShuffle path is used. --- .../spark/serializer/KryoSerializer.scala | 7 ++ .../apache/spark/serializer/Serializer.scala | 35 +++++- .../util/collection/ExternalSorter.scala | 3 +- .../SerializerPropertiesSuite.scala | 119 ++++++++++++++++++ .../sql/execution/SparkSqlSerializer2.scala | 5 + 5 files changed, 166 insertions(+), 3 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/serializer/SerializerPropertiesSuite.scala diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala index b7bc087855b9f..f9f78852f032b 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -125,6 +125,13 @@ class KryoSerializer(conf: SparkConf) override def newInstance(): SerializerInstance = { new KryoSerializerInstance(this) } + + private[spark] override lazy val supportsRelocationOfSerializedObjects: Boolean = { + // If auto-reset is disabled, then Kryo may store references to duplicate occurrences of objects + // in the stream rather than writing those objects' serialized bytes, breaking relocation. See + // https://groups.google.com/d/msg/kryo-users/6ZUSyfjjtdo/FhGG1KHDXPgJ for more details. + newInstance().asInstanceOf[KryoSerializerInstance].getAutoReset() + } } private[spark] diff --git a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala index c381672a4f588..6078c9d433ebf 100644 --- a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala @@ -23,7 +23,7 @@ import java.nio.ByteBuffer import scala.reflect.ClassTag import org.apache.spark.{SparkConf, SparkEnv} -import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.annotation.{DeveloperApi, Private} import org.apache.spark.util.{Utils, ByteBufferInputStream, NextIterator} /** @@ -63,6 +63,39 @@ abstract class Serializer { /** Creates a new [[SerializerInstance]]. */ def newInstance(): SerializerInstance + + /** + * :: Private :: + * Returns true if this serializer supports relocation of its serialized objects and false + * otherwise. This should return true if and only if reordering the bytes of serialized objects + * in serialization stream output is equivalent to having re-ordered those elements prior to + * serializing them. More specifically, the following should hold if a serializer supports + * relocation: + * + * {{{ + * serOut.open() + * position = 0 + * serOut.write(obj1) + * serOut.flush() + * position = # of bytes writen to stream so far + * obj1Bytes = output[0:position-1] + * serOut.write(obj2) + * serOut.flush() + * position2 = # of bytes written to stream so far + * obj2Bytes = output[position:position2-1] + * serIn.open([obj2bytes] concatenate [obj1bytes]) should return (obj2, obj1) + * }}} + * + * In general, this property should hold for serializers that are stateless and that do not + * write special metadata at the beginning or end of the serialization stream. + * + * This API is private to Spark; this method should not be overridden in third-party subclasses + * or called in user code and is subject to removal in future Spark releases. + * + * See SPARK-7311 for more details. + */ + @Private + private[spark] def supportsRelocationOfSerializedObjects: Boolean = false } diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala index b7306cd551918..7d5cf7b61e56a 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala @@ -131,8 +131,7 @@ private[spark] class ExternalSorter[K, V, C]( private val kvChunkSize = conf.getInt("spark.shuffle.sort.kvChunkSize", 1 << 22) // 4 MB private val useSerializedPairBuffer = !ordering.isDefined && conf.getBoolean("spark.shuffle.sort.serializeMapOutputs", true) && - ser.isInstanceOf[KryoSerializer] && - serInstance.asInstanceOf[KryoSerializerInstance].getAutoReset + ser.supportsRelocationOfSerializedObjects // Data structures to store in-memory objects before we spill. Depending on whether we have an // Aggregator set, we either put objects into an AppendOnlyMap where we combine them, or we diff --git a/core/src/test/scala/org/apache/spark/serializer/SerializerPropertiesSuite.scala b/core/src/test/scala/org/apache/spark/serializer/SerializerPropertiesSuite.scala new file mode 100644 index 0000000000000..bb34033fe9e7e --- /dev/null +++ b/core/src/test/scala/org/apache/spark/serializer/SerializerPropertiesSuite.scala @@ -0,0 +1,119 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.serializer + +import java.io.{ByteArrayInputStream, ByteArrayOutputStream} + +import scala.util.Random + +import org.scalatest.{Assertions, FunSuite} + +import org.apache.spark.SparkConf +import org.apache.spark.serializer.KryoTest.RegistratorWithoutAutoReset + +/** + * Tests to ensure that [[Serializer]] implementations obey the API contracts for methods that + * describe properties of the serialized stream, such as + * [[Serializer.supportsRelocationOfSerializedObjects]]. + */ +class SerializerPropertiesSuite extends FunSuite { + + import SerializerPropertiesSuite._ + + test("JavaSerializer does not support relocation") { + // Per a comment on the SPARK-4550 JIRA ticket, Java serialization appears to write out the + // full class name the first time an object is written to an output stream, but subsequent + // references to the class write a more compact identifier; this prevents relocation. + val ser = new JavaSerializer(new SparkConf()) + testSupportsRelocationOfSerializedObjects(ser, generateRandomItem) + } + + test("KryoSerializer supports relocation when auto-reset is enabled") { + val ser = new KryoSerializer(new SparkConf) + assert(ser.newInstance().asInstanceOf[KryoSerializerInstance].getAutoReset()) + testSupportsRelocationOfSerializedObjects(ser, generateRandomItem) + } + + test("KryoSerializer does not support relocation when auto-reset is disabled") { + val conf = new SparkConf().set("spark.kryo.registrator", + classOf[RegistratorWithoutAutoReset].getName) + val ser = new KryoSerializer(conf) + assert(!ser.newInstance().asInstanceOf[KryoSerializerInstance].getAutoReset()) + testSupportsRelocationOfSerializedObjects(ser, generateRandomItem) + } + +} + +object SerializerPropertiesSuite extends Assertions { + + def generateRandomItem(rand: Random): Any = { + val randomFunctions: Seq[() => Any] = Seq( + () => rand.nextInt(), + () => rand.nextString(rand.nextInt(10)), + () => rand.nextDouble(), + () => rand.nextBoolean(), + () => (rand.nextInt(), rand.nextString(rand.nextInt(10))), + () => MyCaseClass(rand.nextInt(), rand.nextString(rand.nextInt(10))), + () => { + val x = MyCaseClass(rand.nextInt(), rand.nextString(rand.nextInt(10))) + (x, x) + } + ) + randomFunctions(rand.nextInt(randomFunctions.size)).apply() + } + + def testSupportsRelocationOfSerializedObjects( + serializer: Serializer, + generateRandomItem: Random => Any): Unit = { + if (!serializer.supportsRelocationOfSerializedObjects) { + return + } + val NUM_TRIALS = 5 + val rand = new Random(42) + for (_ <- 1 to NUM_TRIALS) { + val items = { + // Make sure that we have duplicate occurrences of the same object in the stream: + val randomItems = Seq.fill(10)(generateRandomItem(rand)) + randomItems ++ randomItems.take(5) + } + val baos = new ByteArrayOutputStream() + val serStream = serializer.newInstance().serializeStream(baos) + def serializeItem(item: Any): Array[Byte] = { + val itemStartOffset = baos.toByteArray.length + serStream.writeObject(item) + serStream.flush() + val itemEndOffset = baos.toByteArray.length + baos.toByteArray.slice(itemStartOffset, itemEndOffset).clone() + } + val itemsAndSerializedItems: Seq[(Any, Array[Byte])] = { + val serItems = items.map { + item => (item, serializeItem(item)) + } + serStream.close() + rand.shuffle(serItems) + } + val reorderedSerializedData: Array[Byte] = itemsAndSerializedItems.flatMap(_._2).toArray + val deserializedItemsStream = serializer.newInstance().deserializeStream( + new ByteArrayInputStream(reorderedSerializedData)) + assert(deserializedItemsStream.asIterator.toSeq === itemsAndSerializedItems.map(_._1)) + deserializedItemsStream.close() + } + } +} + +private case class MyCaseClass(foo: Int, bar: String) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer2.scala index 9552f41115866..35ad987eb1a63 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer2.scala @@ -154,6 +154,11 @@ private[sql] class SparkSqlSerializer2(keySchema: Array[DataType], valueSchema: with Serializable{ def newInstance(): SerializerInstance = new ShuffleSerializerInstance(keySchema, valueSchema) + + override def supportsRelocationOfSerializedObjects: Boolean = { + // SparkSqlSerializer2 is stateless and writes no stream headers + true + } } private[sql] object SparkSqlSerializer2 { From 845d1d4d0cc5512d1671b2f2c7c74c5b97c3aef8 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Wed, 6 May 2015 11:03:17 -0700 Subject: [PATCH 14/75] Add `Private` annotation. This was originally added as part of #4435, which was reverted. --- .../org/apache/spark/annotation/Private.java | 41 +++++++++++++++++++ 1 file changed, 41 insertions(+) create mode 100644 core/src/main/scala/org/apache/spark/annotation/Private.java diff --git a/core/src/main/scala/org/apache/spark/annotation/Private.java b/core/src/main/scala/org/apache/spark/annotation/Private.java new file mode 100644 index 0000000000000..9082fcf0c84bc --- /dev/null +++ b/core/src/main/scala/org/apache/spark/annotation/Private.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.annotation; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + * A class that is considered private to the internals of Spark -- there is a high-likelihood + * they will be changed in future versions of Spark. + * + * This should be used only when the standard Scala / Java means of protecting classes are + * insufficient. In particular, Java has no equivalent of private[spark], so we use this annotation + * in its place. + * + * NOTE: If there exists a Scaladoc comment that immediately precedes this annotation, the first + * line of the comment must be ":: Private ::" with no trailing blank line. This is because + * of the known issue that Scaladoc displays only either the annotation or the comment, whichever + * comes first. + */ +@Retention(RetentionPolicy.RUNTIME) +@Target({ElementType.TYPE, ElementType.FIELD, ElementType.METHOD, ElementType.PARAMETER, + ElementType.CONSTRUCTOR, ElementType.LOCAL_VARIABLE, ElementType.PACKAGE}) +public @interface Private {} From 77409967008415b7ea57e9349d296350e6519687 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Wed, 6 May 2015 14:48:25 -0700 Subject: [PATCH 15/75] [HOT-FIX] Move HiveWindowFunctionQuerySuite.scala to hive compatibility dir. Author: Yin Huai Closes #5951 from yhuai/fixBuildMaven and squashes the following commits: fdde183 [Yin Huai] Move HiveWindowFunctionQuerySuite.scala to hive compatibility dir. --- .../spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename sql/hive/{ => compatibility}/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala (100%) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala similarity index 100% rename from sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala rename to sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala From 1ad04dae038673a448f529c39b17817b78d6acd0 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Wed, 6 May 2015 16:15:51 -0700 Subject: [PATCH 16/75] [SPARK-5995] [ML] Make Prediction dev API public Changes: * Update protected prediction methods, following design doc. **<--most interesting change** * Changed abstract classes for Estimator and Model to be public. Added DeveloperApi tag. (I kept the traits for Estimator/Model Params private.) * Changed ProbabilisticClassificationModel method names to use probability instead of probabilities. CC: mengxr shivaram etrain Author: Joseph K. Bradley Closes #5913 from jkbradley/public-dev-api and squashes the following commits: e9aa0ea [Joseph K. Bradley] moved findMax to DenseVector and renamed to argmax. fixed bug for vector of length 0 15b9957 [Joseph K. Bradley] renamed probabilities to probability in method names 5cda84d [Joseph K. Bradley] regenerated sharedParams 7d1877a [Joseph K. Bradley] Made spark.ml prediction abstractions public. Organized their prediction methods for efficient computation of multiple output columns. --- .../ml/{impl/estimator => }/Predictor.scala | 50 ++------ .../spark/ml/classification/Classifier.scala | 110 ++++++------------ .../DecisionTreeClassifier.scala | 5 +- .../ml/classification/GBTClassifier.scala | 5 +- .../classification/LogisticRegression.scala | 100 ++++++---------- .../ProbabilisticClassifier.scala | 100 +++++++++++----- .../RandomForestClassifier.scala | 5 +- .../ml/param/shared/SharedParamsCodeGen.scala | 6 +- .../spark/ml/param/shared/sharedParams.scala | 4 +- .../ml/regression/DecisionTreeRegressor.scala | 5 +- .../spark/ml/regression/GBTRegressor.scala | 5 +- .../ml/regression/LinearRegression.scala | 5 +- .../ml/regression/RandomForestRegressor.scala | 5 +- .../spark/ml/regression/Regressor.scala | 42 ++----- .../spark/ml/{impl => }/tree/treeParams.scala | 4 +- .../apache/spark/mllib/linalg/Vectors.scala | 22 ++++ 16 files changed, 206 insertions(+), 267 deletions(-) rename mllib/src/main/scala/org/apache/spark/ml/{impl/estimator => }/Predictor.scala (86%) rename mllib/src/main/scala/org/apache/spark/ml/{impl => }/tree/treeParams.scala (99%) diff --git a/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala b/mllib/src/main/scala/org/apache/spark/ml/Predictor.scala similarity index 86% rename from mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala rename to mllib/src/main/scala/org/apache/spark/ml/Predictor.scala index e8b3628140e99..0e53877de92db 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Predictor.scala @@ -15,29 +15,23 @@ * limitations under the License. */ -package org.apache.spark.ml.impl.estimator +package org.apache.spark.ml -import org.apache.spark.annotation.{AlphaComponent, DeveloperApi} -import org.apache.spark.ml.{Estimator, Model} +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util.SchemaUtils import org.apache.spark.mllib.linalg.{Vector, VectorUDT} import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{DataFrame, Row} import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.{DataType, DoubleType, StructType} +import org.apache.spark.sql.{DataFrame, Row} /** - * :: DeveloperApi :: - * - * Trait for parameters for prediction (regression and classification). - * - * NOTE: This is currently private[spark] but will be made public later once it is stabilized. + * (private[ml]) Trait for parameters for prediction (regression and classification). */ -@DeveloperApi -private[spark] trait PredictorParams extends Params +private[ml] trait PredictorParams extends Params with HasLabelCol with HasFeaturesCol with HasPredictionCol { /** @@ -63,7 +57,7 @@ private[spark] trait PredictorParams extends Params } /** - * :: AlphaComponent :: + * :: DeveloperApi :: * * Abstraction for prediction problems (regression and classification). * @@ -73,11 +67,9 @@ private[spark] trait PredictorParams extends Params * parameter to specify the concrete type. * @tparam M Specialization of [[PredictionModel]]. If you subclass this type, use this type * parameter to specify the concrete type for the corresponding model. - * - * NOTE: This is currently private[spark] but will be made public later once it is stabilized. */ -@AlphaComponent -private[spark] abstract class Predictor[ +@DeveloperApi +abstract class Predictor[ FeaturesType, Learner <: Predictor[FeaturesType, Learner, M], M <: PredictionModel[FeaturesType, M]] @@ -104,8 +96,6 @@ private[spark] abstract class Predictor[ } /** - * :: DeveloperApi :: - * * Train a model using the given dataset and parameters. * Developers can implement this instead of [[fit()]] to avoid dealing with schema validation * and copying parameters into the model. @@ -113,12 +103,9 @@ private[spark] abstract class Predictor[ * @param dataset Training dataset * @return Fitted model */ - @DeveloperApi protected def train(dataset: DataFrame): M /** - * :: DeveloperApi :: - * * Returns the SQL DataType corresponding to the FeaturesType type parameter. * * This is used by [[validateAndTransformSchema()]]. @@ -126,7 +113,6 @@ private[spark] abstract class Predictor[ * * The default value is VectorUDT, but it may be overridden if FeaturesType is not Vector. */ - @DeveloperApi protected def featuresDataType: DataType = new VectorUDT override def transformSchema(schema: StructType): StructType = { @@ -146,7 +132,7 @@ private[spark] abstract class Predictor[ } /** - * :: AlphaComponent :: + * :: DeveloperApi :: * * Abstraction for a model for prediction tasks (regression and classification). * @@ -154,11 +140,9 @@ private[spark] abstract class Predictor[ * E.g., [[org.apache.spark.mllib.linalg.VectorUDT]] for vector features. * @tparam M Specialization of [[PredictionModel]]. If you subclass this type, use this type * parameter to specify the concrete type for the corresponding model. - * - * NOTE: This is currently private[spark] but will be made public later once it is stabilized. */ -@AlphaComponent -private[spark] abstract class PredictionModel[FeaturesType, M <: PredictionModel[FeaturesType, M]] +@DeveloperApi +abstract class PredictionModel[FeaturesType, M <: PredictionModel[FeaturesType, M]] extends Model[M] with PredictorParams { /** @group setParam */ @@ -168,8 +152,6 @@ private[spark] abstract class PredictionModel[FeaturesType, M <: PredictionModel def setPredictionCol(value: String): M = set(predictionCol, value).asInstanceOf[M] /** - * :: DeveloperApi :: - * * Returns the SQL DataType corresponding to the FeaturesType type parameter. * * This is used by [[validateAndTransformSchema()]]. @@ -177,7 +159,6 @@ private[spark] abstract class PredictionModel[FeaturesType, M <: PredictionModel * * The default value is VectorUDT, but it may be overridden if FeaturesType is not Vector. */ - @DeveloperApi protected def featuresDataType: DataType = new VectorUDT override def transformSchema(schema: StructType): StructType = { @@ -192,12 +173,8 @@ private[spark] abstract class PredictionModel[FeaturesType, M <: PredictionModel * @return transformed dataset with [[predictionCol]] of type [[Double]] */ override def transform(dataset: DataFrame): DataFrame = { - // This default implementation should be overridden as needed. - - // Check schema transformSchema(dataset.schema, logging = true) - - if ($(predictionCol) != "") { + if ($(predictionCol).nonEmpty) { dataset.withColumn($(predictionCol), callUDF(predict _, DoubleType, col($(featuresCol)))) } else { this.logWarning(s"$uid: Predictor.transform() was called as NOOP" + @@ -207,11 +184,8 @@ private[spark] abstract class PredictionModel[FeaturesType, M <: PredictionModel } /** - * :: DeveloperApi :: - * * Predict label for the given features. * This internal method is used to implement [[transform()]] and output [[predictionCol]]. */ - @DeveloperApi protected def predict(features: FeaturesType): Double } diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala index d3361e24705c8..263d580fe2dd3 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala @@ -17,8 +17,8 @@ package org.apache.spark.ml.classification -import org.apache.spark.annotation.{AlphaComponent, DeveloperApi} -import org.apache.spark.ml.impl.estimator.{PredictionModel, Predictor, PredictorParams} +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.ml.{PredictionModel, PredictorParams, Predictor} import org.apache.spark.ml.param.shared.HasRawPredictionCol import org.apache.spark.ml.util.SchemaUtils import org.apache.spark.mllib.linalg.{Vector, VectorUDT} @@ -26,15 +26,12 @@ import org.apache.spark.sql.DataFrame import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.{DataType, DoubleType, StructType} + /** - * :: DeveloperApi :: - * Params for classification. - * - * NOTE: This is currently private[spark] but will be made public later once it is stabilized. + * (private[spark]) Params for classification. */ -@DeveloperApi -private[spark] trait ClassifierParams extends PredictorParams - with HasRawPredictionCol { +private[spark] trait ClassifierParams + extends PredictorParams with HasRawPredictionCol { override protected def validateAndTransformSchema( schema: StructType, @@ -46,23 +43,21 @@ private[spark] trait ClassifierParams extends PredictorParams } /** - * :: AlphaComponent :: + * :: DeveloperApi :: + * * Single-label binary or multiclass classification. * Classes are indexed {0, 1, ..., numClasses - 1}. * * @tparam FeaturesType Type of input features. E.g., [[Vector]] * @tparam E Concrete Estimator type * @tparam M Concrete Model type - * - * NOTE: This is currently private[spark] but will be made public later once it is stabilized. */ -@AlphaComponent -private[spark] abstract class Classifier[ +@DeveloperApi +abstract class Classifier[ FeaturesType, E <: Classifier[FeaturesType, E, M], M <: ClassificationModel[FeaturesType, M]] - extends Predictor[FeaturesType, E, M] - with ClassifierParams { + extends Predictor[FeaturesType, E, M] with ClassifierParams { /** @group setParam */ def setRawPredictionCol(value: String): E = set(rawPredictionCol, value).asInstanceOf[E] @@ -71,17 +66,15 @@ private[spark] abstract class Classifier[ } /** - * :: AlphaComponent :: + * :: DeveloperApi :: + * * Model produced by a [[Classifier]]. * Classes are indexed {0, 1, ..., numClasses - 1}. * * @tparam FeaturesType Type of input features. E.g., [[Vector]] * @tparam M Concrete Model type - * - * NOTE: This is currently private[spark] but will be made public later once it is stabilized. */ -@AlphaComponent -private[spark] +@DeveloperApi abstract class ClassificationModel[FeaturesType, M <: ClassificationModel[FeaturesType, M]] extends PredictionModel[FeaturesType, M] with ClassifierParams { @@ -101,13 +94,27 @@ abstract class ClassificationModel[FeaturesType, M <: ClassificationModel[Featur * @return transformed dataset */ override def transform(dataset: DataFrame): DataFrame = { - // This default implementation should be overridden as needed. - - // Check schema transformSchema(dataset.schema, logging = true) - val (numColsOutput, outputData) = - ClassificationModel.transformColumnsImpl[FeaturesType](dataset, this) + // Output selected columns only. + // This is a bit complicated since it tries to avoid repeated computation. + var outputData = dataset + var numColsOutput = 0 + if (getRawPredictionCol != "") { + outputData = outputData.withColumn(getRawPredictionCol, + callUDF(predictRaw _, new VectorUDT, col(getFeaturesCol))) + numColsOutput += 1 + } + if (getPredictionCol != "") { + val predUDF = if (getRawPredictionCol != "") { + callUDF(raw2prediction _, DoubleType, col(getRawPredictionCol)) + } else { + callUDF(predict _, DoubleType, col(getFeaturesCol)) + } + outputData = outputData.withColumn(getPredictionCol, predUDF) + numColsOutput += 1 + } + if (numColsOutput == 0) { logWarning(s"$uid: ClassificationModel.transform() was called as NOOP" + " since no output columns were set.") @@ -116,22 +123,17 @@ abstract class ClassificationModel[FeaturesType, M <: ClassificationModel[Featur } /** - * :: DeveloperApi :: - * * Predict label for the given features. * This internal method is used to implement [[transform()]] and output [[predictionCol]]. * * This default implementation for classification predicts the index of the maximum value * from [[predictRaw()]]. */ - @DeveloperApi override protected def predict(features: FeaturesType): Double = { - predictRaw(features).toArray.zipWithIndex.maxBy(_._1)._2 + raw2prediction(predictRaw(features)) } /** - * :: DeveloperApi :: - * * Raw prediction for each possible label. * The meaning of a "raw" prediction may vary between algorithms, but it intuitively gives * a measure of confidence in each possible label (where larger = more confident). @@ -141,48 +143,12 @@ abstract class ClassificationModel[FeaturesType, M <: ClassificationModel[Featur * This raw prediction may be any real number, where a larger value indicates greater * confidence for that label. */ - @DeveloperApi protected def predictRaw(features: FeaturesType): Vector -} - -private[ml] object ClassificationModel { /** - * Added prediction column(s). This is separated from [[ClassificationModel.transform()]] - * since it is used by [[org.apache.spark.ml.classification.ProbabilisticClassificationModel]]. - * @param dataset Input dataset - * @return (number of columns added, transformed dataset) + * Given a vector of raw predictions, select the predicted label. + * This may be overridden to support thresholds which favor particular labels. + * @return predicted label */ - def transformColumnsImpl[FeaturesType]( - dataset: DataFrame, - model: ClassificationModel[FeaturesType, _]): (Int, DataFrame) = { - - // Output selected columns only. - // This is a bit complicated since it tries to avoid repeated computation. - var tmpData = dataset - var numColsOutput = 0 - if (model.getRawPredictionCol != "") { - // output raw prediction - val features2raw: FeaturesType => Vector = model.predictRaw - tmpData = tmpData.withColumn(model.getRawPredictionCol, - callUDF(features2raw, new VectorUDT, col(model.getFeaturesCol))) - numColsOutput += 1 - if (model.getPredictionCol != "") { - val raw2pred: Vector => Double = (rawPred) => { - rawPred.toArray.zipWithIndex.maxBy(_._1)._2 - } - tmpData = tmpData.withColumn(model.getPredictionCol, - callUDF(raw2pred, DoubleType, col(model.getRawPredictionCol))) - numColsOutput += 1 - } - } else if (model.getPredictionCol != "") { - // output prediction - val features2pred: FeaturesType => Double = model.predict - tmpData = tmpData.withColumn(model.getPredictionCol, - callUDF(features2pred, DoubleType, col(model.getFeaturesCol))) - numColsOutput += 1 - } - (numColsOutput, tmpData) - } - + protected def raw2prediction(rawPrediction: Vector): Double = rawPrediction.toDense.argmax } diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala index 419e5ba05d38a..dcebea1d4b015 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala @@ -18,10 +18,9 @@ package org.apache.spark.ml.classification import org.apache.spark.annotation.AlphaComponent -import org.apache.spark.ml.impl.estimator.{PredictionModel, Predictor} -import org.apache.spark.ml.impl.tree._ +import org.apache.spark.ml.{PredictionModel, Predictor} import org.apache.spark.ml.param.ParamMap -import org.apache.spark.ml.tree.{DecisionTreeModel, Node} +import org.apache.spark.ml.tree.{TreeClassifierParams, DecisionTreeParams, DecisionTreeModel, Node} import org.apache.spark.ml.util.MetadataUtils import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.regression.LabeledPoint diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala index 534ea95b1c538..ae51b05a0c42d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala @@ -21,11 +21,10 @@ import com.github.fommil.netlib.BLAS.{getInstance => blas} import org.apache.spark.Logging import org.apache.spark.annotation.AlphaComponent -import org.apache.spark.ml.impl.estimator.{PredictionModel, Predictor} -import org.apache.spark.ml.impl.tree._ +import org.apache.spark.ml.{PredictionModel, Predictor} import org.apache.spark.ml.param.{Param, ParamMap} import org.apache.spark.ml.regression.DecisionTreeRegressionModel -import org.apache.spark.ml.tree.{DecisionTreeModel, TreeEnsembleModel} +import org.apache.spark.ml.tree.{GBTParams, TreeClassifierParams, DecisionTreeModel, TreeEnsembleModel} import org.apache.spark.ml.util.MetadataUtils import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.regression.LabeledPoint diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala index b73be035e29b5..550369d18cfec 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala @@ -21,9 +21,8 @@ import org.apache.spark.annotation.AlphaComponent import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ import org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS -import org.apache.spark.mllib.linalg.{BLAS, Vector, VectorUDT, Vectors} +import org.apache.spark.mllib.linalg._ import org.apache.spark.sql.DataFrame -import org.apache.spark.sql.functions._ import org.apache.spark.storage.StorageLevel /** @@ -99,76 +98,17 @@ class LogisticRegressionModel private[ml] ( /** @group setParam */ def setThreshold(value: Double): this.type = set(threshold, value) + /** Margin (rawPrediction) for class label 1. For binary classification only. */ private val margin: Vector => Double = (features) => { BLAS.dot(features, weights) + intercept } + /** Score (probability) for class label 1. For binary classification only. */ private val score: Vector => Double = (features) => { val m = margin(features) 1.0 / (1.0 + math.exp(-m)) } - override def transform(dataset: DataFrame): DataFrame = { - // This is overridden (a) to be more efficient (avoiding re-computing values when creating - // multiple output columns) and (b) to handle threshold, which the abstractions do not use. - // TODO: We should abstract away the steps defined by UDFs below so that the abstractions - // can call whichever UDFs are needed to create the output columns. - - // Check schema - transformSchema(dataset.schema, logging = true) - - // Output selected columns only. - // This is a bit complicated since it tries to avoid repeated computation. - // rawPrediction (-margin, margin) - // probability (1.0-score, score) - // prediction (max margin) - var tmpData = dataset - var numColsOutput = 0 - if ($(rawPredictionCol) != "") { - val features2raw: Vector => Vector = (features) => predictRaw(features) - tmpData = tmpData.withColumn($(rawPredictionCol), - callUDF(features2raw, new VectorUDT, col($(featuresCol)))) - numColsOutput += 1 - } - if ($(probabilityCol) != "") { - if ($(rawPredictionCol) != "") { - val raw2prob = udf { (rawPreds: Vector) => - val prob1 = 1.0 / (1.0 + math.exp(-rawPreds(1))) - Vectors.dense(1.0 - prob1, prob1): Vector - } - tmpData = tmpData.withColumn($(probabilityCol), raw2prob(col($(rawPredictionCol)))) - } else { - val features2prob = udf { (features: Vector) => predictProbabilities(features) : Vector } - tmpData = tmpData.withColumn($(probabilityCol), features2prob(col($(featuresCol)))) - } - numColsOutput += 1 - } - if ($(predictionCol) != "") { - val t = $(threshold) - if ($(probabilityCol) != "") { - val predict = udf { probs: Vector => - if (probs(1) > t) 1.0 else 0.0 - } - tmpData = tmpData.withColumn($(predictionCol), predict(col($(probabilityCol)))) - } else if ($(rawPredictionCol) != "") { - val predict = udf { rawPreds: Vector => - val prob1 = 1.0 / (1.0 + math.exp(-rawPreds(1))) - if (prob1 > t) 1.0 else 0.0 - } - tmpData = tmpData.withColumn($(predictionCol), predict(col($(rawPredictionCol)))) - } else { - val predict = udf { features: Vector => this.predict(features) } - tmpData = tmpData.withColumn($(predictionCol), predict(col($(featuresCol)))) - } - numColsOutput += 1 - } - if (numColsOutput == 0) { - this.logWarning(s"$uid: LogisticRegressionModel.transform() was called as NOOP" + - " since no output columns were set.") - } - tmpData - } - override val numClasses: Int = 2 /** @@ -179,17 +119,43 @@ class LogisticRegressionModel private[ml] ( if (score(features) > getThreshold) 1 else 0 } - override protected def predictProbabilities(features: Vector): Vector = { - val s = score(features) - Vectors.dense(1.0 - s, s) + override protected def raw2probabilityInPlace(rawPrediction: Vector): Vector = { + rawPrediction match { + case dv: DenseVector => + var i = 0 + while (i < dv.size) { + dv.values(i) = 1.0 / (1.0 + math.exp(-dv.values(i))) + i += 1 + } + dv + case sv: SparseVector => + throw new RuntimeException("Unexpected error in LogisticRegressionModel:" + + " raw2probabilitiesInPlace encountered SparseVector") + } } override protected def predictRaw(features: Vector): Vector = { val m = margin(features) - Vectors.dense(0.0, m) + Vectors.dense(-m, m) } override def copy(extra: ParamMap): LogisticRegressionModel = { copyValues(new LogisticRegressionModel(parent, weights, intercept), extra) } + + override protected def raw2prediction(rawPrediction: Vector): Double = { + val t = getThreshold + val rawThreshold = if (t == 0.0) { + Double.NegativeInfinity + } else if (t == 1.0) { + Double.PositiveInfinity + } else { + Math.log(t / (1.0 - t)) + } + if (rawPrediction(1) > rawThreshold) 1 else 0 + } + + override protected def probability2prediction(probability: Vector): Double = { + if (probability(1) > getThreshold) 1 else 0 + } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala index 8519841c5c26c..330ae2938f4e0 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala @@ -17,16 +17,16 @@ package org.apache.spark.ml.classification -import org.apache.spark.annotation.{AlphaComponent, DeveloperApi} +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util.SchemaUtils import org.apache.spark.mllib.linalg.{Vector, VectorUDT} import org.apache.spark.sql.DataFrame import org.apache.spark.sql.functions._ -import org.apache.spark.sql.types.{DataType, StructType} +import org.apache.spark.sql.types.{DoubleType, DataType, StructType} /** - * Params for probabilistic classification. + * (private[classification]) Params for probabilistic classification. */ private[classification] trait ProbabilisticClassifierParams extends ClassifierParams with HasProbabilityCol { @@ -42,17 +42,15 @@ private[classification] trait ProbabilisticClassifierParams /** - * :: AlphaComponent :: + * :: DeveloperApi :: * * Single-label binary or multiclass classifier which can output class conditional probabilities. * * @tparam FeaturesType Type of input features. E.g., [[Vector]] * @tparam E Concrete Estimator type * @tparam M Concrete Model type - * - * NOTE: This is currently private[spark] but will be made public later once it is stabilized. */ -@AlphaComponent +@DeveloperApi private[spark] abstract class ProbabilisticClassifier[ FeaturesType, E <: ProbabilisticClassifier[FeaturesType, E, M], @@ -65,17 +63,15 @@ private[spark] abstract class ProbabilisticClassifier[ /** - * :: AlphaComponent :: + * :: DeveloperApi :: * * Model produced by a [[ProbabilisticClassifier]]. * Classes are indexed {0, 1, ..., numClasses - 1}. * * @tparam FeaturesType Type of input features. E.g., [[Vector]] * @tparam M Concrete Model type - * - * NOTE: This is currently private[spark] but will be made public later once it is stabilized. */ -@AlphaComponent +@DeveloperApi private[spark] abstract class ProbabilisticClassificationModel[ FeaturesType, M <: ProbabilisticClassificationModel[FeaturesType, M]] @@ -95,39 +91,79 @@ private[spark] abstract class ProbabilisticClassificationModel[ * @return transformed dataset */ override def transform(dataset: DataFrame): DataFrame = { - // This default implementation should be overridden as needed. - - // Check schema transformSchema(dataset.schema, logging = true) - val (numColsOutput, outputData) = - ClassificationModel.transformColumnsImpl[FeaturesType](dataset, this) - // Output selected columns only. - if ($(probabilityCol) != "") { - // output probabilities - outputData.withColumn($(probabilityCol), - callUDF(predictProbabilities _, new VectorUDT, col($(featuresCol)))) - } else { - if (numColsOutput == 0) { - this.logWarning(s"$uid: ProbabilisticClassificationModel.transform() was called as NOOP" + - " since no output columns were set.") + // This is a bit complicated since it tries to avoid repeated computation. + var outputData = dataset + var numColsOutput = 0 + if ($(rawPredictionCol).nonEmpty) { + outputData = outputData.withColumn(getRawPredictionCol, + callUDF(predictRaw _, new VectorUDT, col(getFeaturesCol))) + numColsOutput += 1 + } + if ($(probabilityCol).nonEmpty) { + val probUDF = if ($(rawPredictionCol).nonEmpty) { + callUDF(raw2probability _, new VectorUDT, col($(rawPredictionCol))) + } else { + callUDF(predictProbability _, new VectorUDT, col($(featuresCol))) + } + outputData = outputData.withColumn($(probabilityCol), probUDF) + numColsOutput += 1 + } + if ($(predictionCol).nonEmpty) { + val predUDF = if ($(rawPredictionCol).nonEmpty) { + callUDF(raw2prediction _, DoubleType, col($(rawPredictionCol))) + } else if ($(probabilityCol).nonEmpty) { + callUDF(probability2prediction _, DoubleType, col($(probabilityCol))) + } else { + callUDF(predict _, DoubleType, col($(featuresCol))) } - outputData + outputData = outputData.withColumn($(predictionCol), predUDF) + numColsOutput += 1 + } + + if (numColsOutput == 0) { + this.logWarning(s"$uid: ProbabilisticClassificationModel.transform() was called as NOOP" + + " since no output columns were set.") } + outputData } /** - * :: DeveloperApi :: + * Estimate the probability of each class given the raw prediction, + * doing the computation in-place. + * These predictions are also called class conditional probabilities. + * + * This internal method is used to implement [[transform()]] and output [[probabilityCol]]. * + * @return Estimated class conditional probabilities (modified input vector) + */ + protected def raw2probabilityInPlace(rawPrediction: Vector): Vector + + /** Non-in-place version of [[raw2probabilityInPlace()]] */ + protected def raw2probability(rawPrediction: Vector): Vector = { + val probs = rawPrediction.copy + raw2probabilityInPlace(probs) + } + + /** * Predict the probability of each class given the features. * These predictions are also called class conditional probabilities. * - * WARNING: Not all models output well-calibrated probability estimates! These probabilities - * should be treated as confidences, not precise probabilities. - * * This internal method is used to implement [[transform()]] and output [[probabilityCol]]. + * + * @return Estimated class conditional probabilities + */ + protected def predictProbability(features: FeaturesType): Vector = { + val rawPreds = predictRaw(features) + raw2probabilityInPlace(rawPreds) + } + + /** + * Given a vector of class conditional probabilities, select the predicted label. + * This may be overridden to support thresholds which favor particular labels. + * @return predicted label */ - @DeveloperApi - protected def predictProbabilities(features: FeaturesType): Vector + protected def probability2prediction(probability: Vector): Double = probability.toDense.argmax } diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala index 17f59bb42e129..9954893f14359 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala @@ -20,10 +20,9 @@ package org.apache.spark.ml.classification import scala.collection.mutable import org.apache.spark.annotation.AlphaComponent -import org.apache.spark.ml.impl.estimator.{PredictionModel, Predictor} -import org.apache.spark.ml.impl.tree._ +import org.apache.spark.ml.{PredictionModel, Predictor} import org.apache.spark.ml.param.ParamMap -import org.apache.spark.ml.tree.{DecisionTreeModel, TreeEnsembleModel} +import org.apache.spark.ml.tree.{RandomForestParams, TreeClassifierParams, DecisionTreeModel, TreeEnsembleModel} import org.apache.spark.ml.util.MetadataUtils import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.regression.LabeledPoint diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/shared/SharedParamsCodeGen.scala b/mllib/src/main/scala/org/apache/spark/ml/param/shared/SharedParamsCodeGen.scala index d379172e0bf53..0e1ff97a8bf60 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/param/shared/SharedParamsCodeGen.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/param/shared/SharedParamsCodeGen.scala @@ -40,8 +40,10 @@ private[shared] object SharedParamsCodeGen { ParamDesc[String]("predictionCol", "prediction column name", Some("\"prediction\"")), ParamDesc[String]("rawPredictionCol", "raw prediction (a.k.a. confidence) column name", Some("\"rawPrediction\"")), - ParamDesc[String]("probabilityCol", - "column name for predicted class conditional probabilities", Some("\"probability\"")), + ParamDesc[String]("probabilityCol", "Column name for predicted class conditional" + + " probabilities. Note: Not all models output well-calibrated probability estimates!" + + " These probabilities should be treated as confidences, not precise probabilities.", + Some("\"probability\"")), ParamDesc[Double]("threshold", "threshold in binary classification prediction, in range [0, 1]", isValid = "ParamValidators.inRange(0, 1)"), diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala b/mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala index fb1874ccfc8dc..87f86807c3c91 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala @@ -128,10 +128,10 @@ private[ml] trait HasRawPredictionCol extends Params { private[ml] trait HasProbabilityCol extends Params { /** - * Param for column name for predicted class conditional probabilities. + * Param for Column name for predicted class conditional probabilities. Note: Not all models output well-calibrated probability estimates! These probabilities should be treated as confidences, not precise probabilities.. * @group param */ - final val probabilityCol: Param[String] = new Param[String](this, "probabilityCol", "column name for predicted class conditional probabilities") + final val probabilityCol: Param[String] = new Param[String](this, "probabilityCol", "Column name for predicted class conditional probabilities. Note: Not all models output well-calibrated probability estimates! These probabilities should be treated as confidences, not precise probabilities.") setDefault(probabilityCol, "probability") diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala index b07c26fe79b36..f8f0b161a4812 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala @@ -18,10 +18,9 @@ package org.apache.spark.ml.regression import org.apache.spark.annotation.AlphaComponent -import org.apache.spark.ml.impl.estimator.{PredictionModel, Predictor} -import org.apache.spark.ml.impl.tree._ +import org.apache.spark.ml.{PredictionModel, Predictor} import org.apache.spark.ml.param.ParamMap -import org.apache.spark.ml.tree.{DecisionTreeModel, Node} +import org.apache.spark.ml.tree.{TreeRegressorParams, DecisionTreeParams, DecisionTreeModel, Node} import org.apache.spark.ml.util.MetadataUtils import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.regression.LabeledPoint diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala index bc796958e4545..461905c12701a 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala @@ -21,10 +21,9 @@ import com.github.fommil.netlib.BLAS.{getInstance => blas} import org.apache.spark.Logging import org.apache.spark.annotation.AlphaComponent -import org.apache.spark.ml.impl.estimator.{PredictionModel, Predictor} -import org.apache.spark.ml.impl.tree._ +import org.apache.spark.ml.{PredictionModel, Predictor} import org.apache.spark.ml.param.{Param, ParamMap} -import org.apache.spark.ml.tree.{DecisionTreeModel, TreeEnsembleModel} +import org.apache.spark.ml.tree.{GBTParams, TreeRegressorParams, DecisionTreeModel, TreeEnsembleModel} import org.apache.spark.ml.util.MetadataUtils import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.regression.LabeledPoint diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala index 66c475f2d9840..e63c9a3eead52 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala @@ -25,6 +25,7 @@ import breeze.optimize.{CachedDiffFunction, DiffFunction, LBFGS => BreezeLBFGS, import org.apache.spark.Logging import org.apache.spark.annotation.AlphaComponent +import org.apache.spark.ml.PredictorParams import org.apache.spark.ml.param.ParamMap import org.apache.spark.ml.param.shared.{HasElasticNetParam, HasMaxIter, HasRegParam, HasTol} import org.apache.spark.mllib.linalg.{Vector, Vectors} @@ -39,7 +40,7 @@ import org.apache.spark.util.StatCounter /** * Params for linear regression. */ -private[regression] trait LinearRegressionParams extends RegressorParams +private[regression] trait LinearRegressionParams extends PredictorParams with HasRegParam with HasElasticNetParam with HasMaxIter with HasTol /** @@ -240,7 +241,7 @@ class LinearRegressionModel private[ml] ( * + \bar{y} / \hat{y}||^2 * = 1/2n ||\sum_i w_i^\prime x_i - y / \hat{y} + offset||^2 = 1/2n diff^2 * }}} - * where w_i^\prime is the effective weights defined by w_i/\hat{x_i}, offset is + * where w_i^\prime^ is the effective weights defined by w_i/\hat{x_i}, offset is * {{{ * - \sum_i (w_i/\hat{x_i})\bar{x_i} + \bar{y} / \hat{y}. * }}}, and diff is diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala index 0468a1be1ba74..dbc628927433d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala @@ -18,10 +18,9 @@ package org.apache.spark.ml.regression import org.apache.spark.annotation.AlphaComponent -import org.apache.spark.ml.impl.estimator.{PredictionModel, Predictor} -import org.apache.spark.ml.impl.tree.{RandomForestParams, TreeRegressorParams} +import org.apache.spark.ml.{PredictionModel, Predictor} import org.apache.spark.ml.param.ParamMap -import org.apache.spark.ml.tree.{DecisionTreeModel, TreeEnsembleModel} +import org.apache.spark.ml.tree.{RandomForestParams, TreeRegressorParams, DecisionTreeModel, TreeEnsembleModel} import org.apache.spark.ml.util.MetadataUtils import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.regression.LabeledPoint diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/Regressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/Regressor.scala index c6b3327db6ad3..c72ef29680329 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/Regressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/Regressor.scala @@ -17,62 +17,40 @@ package org.apache.spark.ml.regression -import org.apache.spark.annotation.{AlphaComponent, DeveloperApi} -import org.apache.spark.ml.impl.estimator.{PredictionModel, Predictor, PredictorParams} +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.ml.{PredictionModel, PredictorParams, Predictor} -/** - * :: DeveloperApi :: - * Params for regression. - * Currently empty, but may add functionality later. - * - * NOTE: This is currently private[spark] but will be made public later once it is stabilized. - */ -@DeveloperApi -private[spark] trait RegressorParams extends PredictorParams /** - * :: AlphaComponent :: + * :: DeveloperApi :: * * Single-label regression * * @tparam FeaturesType Type of input features. E.g., [[org.apache.spark.mllib.linalg.Vector]] * @tparam Learner Concrete Estimator type * @tparam M Concrete Model type - * - * NOTE: This is currently private[spark] but will be made public later once it is stabilized. */ -@AlphaComponent +@DeveloperApi private[spark] abstract class Regressor[ FeaturesType, Learner <: Regressor[FeaturesType, Learner, M], M <: RegressionModel[FeaturesType, M]] - extends Predictor[FeaturesType, Learner, M] - with RegressorParams { + extends Predictor[FeaturesType, Learner, M] with PredictorParams { // TODO: defaultEvaluator (follow-up PR) } /** - * :: AlphaComponent :: + * :: DeveloperApi :: * * Model produced by a [[Regressor]]. * * @tparam FeaturesType Type of input features. E.g., [[org.apache.spark.mllib.linalg.Vector]] * @tparam M Concrete Model type. - * - * NOTE: This is currently private[spark] but will be made public later once it is stabilized. */ -@AlphaComponent -private[spark] abstract class RegressionModel[FeaturesType, M <: RegressionModel[FeaturesType, M]] - extends PredictionModel[FeaturesType, M] with RegressorParams { - - /** - * :: DeveloperApi :: - * - * Predict real-valued label for the given features. - * This internal method is used to implement [[transform()]] and output [[predictionCol]]. - */ - @DeveloperApi - protected def predict(features: FeaturesType): Double +@DeveloperApi +abstract class RegressionModel[FeaturesType, M <: RegressionModel[FeaturesType, M]] + extends PredictionModel[FeaturesType, M] with PredictorParams { + // TODO: defaultEvaluator (follow-up PR) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/impl/tree/treeParams.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/treeParams.scala similarity index 99% rename from mllib/src/main/scala/org/apache/spark/ml/impl/tree/treeParams.scala rename to mllib/src/main/scala/org/apache/spark/ml/tree/treeParams.scala index 0e225627d4ee3..816fcedf2efb3 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/impl/tree/treeParams.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/treeParams.scala @@ -15,10 +15,10 @@ * limitations under the License. */ -package org.apache.spark.ml.impl.tree +package org.apache.spark.ml.tree import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.ml.impl.estimator.PredictorParams +import org.apache.spark.ml.PredictorParams import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared.{HasMaxIter, HasSeed} import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo, BoostingStrategy => OldBoostingStrategy, Strategy => OldStrategy} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala index 188d1e542b5b5..f6bcdf83cd337 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala @@ -587,6 +587,28 @@ class DenseVector(val values: Array[Double]) extends Vector { } new SparseVector(size, ii, vv) } + + /** + * Find the index of a maximal element. Returns the first maximal element in case of a tie. + * Returns -1 if vector has length 0. + */ + private[spark] def argmax: Int = { + if (size == 0) { + -1 + } else { + var maxIdx = 0 + var maxValue = values(0) + var i = 1 + while (i < size) { + if (values(i) > maxValue) { + maxIdx = i + maxValue = values(i) + } + i += 1 + } + maxIdx + } + } } object DenseVector { From fbf1f342a02af65f69e0ee770a2b983c69e7c089 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 6 May 2015 17:08:39 -0700 Subject: [PATCH 17/75] [HOT FIX] [SPARK-7418] Ignore flaky SparkSubmitUtilsSuite test --- .../scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala index 2df2597e058cd..da9578478bed9 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala @@ -104,7 +104,7 @@ class SparkSubmitUtilsSuite extends FunSuite with BeforeAndAfterAll { assert(jarPath.indexOf(ivyPath) >= 0, "should use non-default ivy path") } - test("search for artifact at other repositories") { + ignore("search for artifact at other repositories") { val path = SparkSubmitUtils.resolveMavenCoordinates("com.agimatec:agimatec-validation:0.9.3", Option("https://oss.sonatype.org/content/repositories/agimatec/"), None, true) assert(path.indexOf("agimatec-validation") >= 0, "should find package. If it doesn't, check" + From 4e930420c19ae7773b138dfc7db8fc03b4660251 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Wed, 6 May 2015 17:28:11 -0700 Subject: [PATCH 18/75] [SPARK-6799] [SPARKR] Remove SparkR RDD examples, add dataframe examples This PR also makes some of the DataFrame to RDD methods private as the RDD class is private in 1.4 cc rxin pwendell Author: Shivaram Venkataraman Closes #5949 from shivaram/sparkr-examples and squashes the following commits: 6c42fdc [Shivaram Venkataraman] Remove SparkR RDD examples, add dataframe examples --- R/pkg/NAMESPACE | 4 - R/pkg/R/DataFrame.R | 2 +- examples/src/main/r/dataframe.R | 54 +++++++++++ examples/src/main/r/kmeans.R | 93 ------------------- examples/src/main/r/linear_solver_mnist.R | 107 ---------------------- examples/src/main/r/logistic_regression.R | 62 ------------- examples/src/main/r/pi.R | 46 ---------- examples/src/main/r/wordcount.R | 42 --------- 8 files changed, 55 insertions(+), 355 deletions(-) create mode 100644 examples/src/main/r/dataframe.R delete mode 100644 examples/src/main/r/kmeans.R delete mode 100644 examples/src/main/r/linear_solver_mnist.R delete mode 100644 examples/src/main/r/logistic_regression.R delete mode 100644 examples/src/main/r/pi.R delete mode 100644 examples/src/main/r/wordcount.R diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE index 528e6608c3c82..3fb92be0940b7 100644 --- a/R/pkg/NAMESPACE +++ b/R/pkg/NAMESPACE @@ -45,8 +45,6 @@ exportMethods("cache", "showDF", "sortDF", "take", - "toJSON", - "toRDD", "unionAll", "unpersist", "where", @@ -95,14 +93,12 @@ export("cacheTable", "createExternalTable", "dropTempTable", "jsonFile", - "jsonRDD", "loadDF", "parquetFile", "sql", "table", "tableNames", "tables", - "toDF", "uncacheTable") export("sparkRSQL.init", diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index 56c305d912587..47d92f141cc7d 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -272,7 +272,7 @@ setMethod("names", setMethod("registerTempTable", signature(x = "DataFrame", tableName = "character"), function(x, tableName) { - callJMethod(x@sdf, "registerTempTable", tableName) + invisible(callJMethod(x@sdf, "registerTempTable", tableName)) }) #' insertInto diff --git a/examples/src/main/r/dataframe.R b/examples/src/main/r/dataframe.R new file mode 100644 index 0000000000000..53b817144f6ac --- /dev/null +++ b/examples/src/main/r/dataframe.R @@ -0,0 +1,54 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +library(SparkR) + +# Initialize SparkContext and SQLContext +sc <- sparkR.init(appName="SparkR-DataFrame-example") +sqlContext <- sparkRSQL.init(sc) + +# Create a simple local data.frame +localDF <- data.frame(name=c("John", "Smith", "Sarah"), age=c(19, 23, 18)) + +# Convert local data frame to a SparkR DataFrame +df <- createDataFrame(sqlContext, localDF) + +# Print its schema +printSchema(df) +# root +# |-- name: string (nullable = true) +# |-- age: double (nullable = true) + +# Create a DataFrame from a JSON file +path <- file.path(Sys.getenv("SPARK_HOME"), "examples/src/main/resources/people.json") +peopleDF <- jsonFile(sqlContext, path) +printSchema(peopleDF) + +# Register this DataFrame as a table. +registerTempTable(peopleDF, "people") + +# SQL statements can be run by using the sql methods provided by sqlContext +teenagers <- sql(sqlContext, "SELECT name FROM people WHERE age >= 13 AND age <= 19") + +# Call collect to get a local data.frame +teenagersLocalDF <- collect(teenagers) + +# Print the teenagers in our dataset +print(teenagersLocalDF) + +# Stop the SparkContext now +sparkR.stop() diff --git a/examples/src/main/r/kmeans.R b/examples/src/main/r/kmeans.R deleted file mode 100644 index 6e6b5cb93789c..0000000000000 --- a/examples/src/main/r/kmeans.R +++ /dev/null @@ -1,93 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -library(SparkR) - -# Logistic regression in Spark. -# Note: unlike the example in Scala, a point here is represented as a vector of -# doubles. - -parseVectors <- function(lines) { - lines <- strsplit(as.character(lines) , " ", fixed = TRUE) - list(matrix(as.numeric(unlist(lines)), ncol = length(lines[[1]]))) -} - -dist.fun <- function(P, C) { - apply( - C, - 1, - function(x) { - colSums((t(P) - x)^2) - } - ) -} - -closestPoint <- function(P, C) { - max.col(-dist.fun(P, C)) -} -# Main program - -args <- commandArgs(trailing = TRUE) - -if (length(args) != 3) { - print("Usage: kmeans ") - q("no") -} - -sc <- sparkR.init(appName = "RKMeans") -K <- as.integer(args[[2]]) -convergeDist <- as.double(args[[3]]) - -lines <- textFile(sc, args[[1]]) -points <- cache(lapplyPartition(lines, parseVectors)) -# kPoints <- take(points, K) -kPoints <- do.call(rbind, takeSample(points, FALSE, K, 16189L)) -tempDist <- 1.0 - -while (tempDist > convergeDist) { - closest <- lapplyPartition( - lapply(points, - function(p) { - cp <- closestPoint(p, kPoints); - mapply(list, unique(cp), split.data.frame(cbind(1, p), cp), SIMPLIFY=FALSE) - }), - function(x) {do.call(c, x) - }) - - pointStats <- reduceByKey(closest, - function(p1, p2) { - t(colSums(rbind(p1, p2))) - }, - 2L) - - newPoints <- do.call( - rbind, - collect(lapply(pointStats, - function(tup) { - point.sum <- tup[[2]][, -1] - point.count <- tup[[2]][, 1] - point.sum/point.count - }))) - - D <- dist.fun(kPoints, newPoints) - tempDist <- sum(D[cbind(1:3, max.col(-D))]) - kPoints <- newPoints - cat("Finished iteration (delta = ", tempDist, ")\n") -} - -cat("Final centers:\n") -writeLines(unlist(lapply(kPoints, paste, collapse = " "))) diff --git a/examples/src/main/r/linear_solver_mnist.R b/examples/src/main/r/linear_solver_mnist.R deleted file mode 100644 index c864a4232d010..0000000000000 --- a/examples/src/main/r/linear_solver_mnist.R +++ /dev/null @@ -1,107 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -# Instructions: https://github.com/amplab-extras/SparkR-pkg/wiki/SparkR-Example:-Digit-Recognition-on-EC2 - -library(SparkR) -library(Matrix) - -args <- commandArgs(trailing = TRUE) - -# number of random features; default to 1100 -D <- ifelse(length(args) > 0, as.integer(args[[1]]), 1100) -# number of partitions for training dataset -trainParts <- 12 -# dimension of digits -d <- 784 -# number of test examples -NTrain <- 60000 -# number of training examples -NTest <- 10000 -# scale of features -gamma <- 4e-4 - -sc <- sparkR.init(appName = "SparkR-LinearSolver") - -# You can also use HDFS path to speed things up: -# hdfs:///train-mnist-dense-with-labels.data -file <- textFile(sc, "/data/train-mnist-dense-with-labels.data", trainParts) - -W <- gamma * matrix(nrow=D, ncol=d, data=rnorm(D*d)) -b <- 2 * pi * matrix(nrow=D, ncol=1, data=runif(D)) -broadcastW <- broadcast(sc, W) -broadcastB <- broadcast(sc, b) - -includePackage(sc, Matrix) -numericLines <- lapplyPartitionsWithIndex(file, - function(split, part) { - matList <- sapply(part, function(line) { - as.numeric(strsplit(line, ",", fixed=TRUE)[[1]]) - }, simplify=FALSE) - mat <- Matrix(ncol=d+1, data=unlist(matList, F, F), - sparse=T, byrow=T) - mat - }) - -featureLabels <- cache(lapplyPartition( - numericLines, - function(part) { - label <- part[,1] - mat <- part[,-1] - ones <- rep(1, nrow(mat)) - features <- cos( - mat %*% t(value(broadcastW)) + (matrix(ncol=1, data=ones) %*% t(value(broadcastB)))) - onesMat <- Matrix(ones) - featuresPlus <- cBind(features, onesMat) - labels <- matrix(nrow=nrow(mat), ncol=10, data=-1) - for (i in 1:nrow(mat)) { - labels[i, label[i]] <- 1 - } - list(label=labels, features=featuresPlus) - })) - -FTF <- Reduce("+", collect(lapplyPartition(featureLabels, - function(part) { - t(part$features) %*% part$features - }), flatten=F)) - -FTY <- Reduce("+", collect(lapplyPartition(featureLabels, - function(part) { - t(part$features) %*% part$label - }), flatten=F)) - -# solve for the coefficient matrix -C <- solve(FTF, FTY) - -test <- Matrix(as.matrix(read.csv("/data/test-mnist-dense-with-labels.data", - header=F), sparse=T)) -testData <- test[,-1] -testLabels <- matrix(ncol=1, test[,1]) - -err <- 0 - -# contstruct the feature maps for all examples from this digit -featuresTest <- cos(testData %*% t(value(broadcastW)) + - (matrix(ncol=1, data=rep(1, NTest)) %*% t(value(broadcastB)))) -featuresTest <- cBind(featuresTest, Matrix(rep(1, NTest))) - -# extract the one vs. all assignment -results <- featuresTest %*% C -labelsGot <- apply(results, 1, which.max) -err <- sum(testLabels != labelsGot) / nrow(testLabels) - -cat("\nFinished running. The error rate is: ", err, ".\n") diff --git a/examples/src/main/r/logistic_regression.R b/examples/src/main/r/logistic_regression.R deleted file mode 100644 index 2a86aa98160d3..0000000000000 --- a/examples/src/main/r/logistic_regression.R +++ /dev/null @@ -1,62 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -library(SparkR) - -args <- commandArgs(trailing = TRUE) - -if (length(args) != 3) { - print("Usage: logistic_regression ") - q("no") -} - -# Initialize Spark context -sc <- sparkR.init(appName = "LogisticRegressionR") -iterations <- as.integer(args[[2]]) -D <- as.integer(args[[3]]) - -readPartition <- function(part){ - part = strsplit(part, " ", fixed = T) - list(matrix(as.numeric(unlist(part)), ncol = length(part[[1]]))) -} - -# Read data points and convert each partition to a matrix -points <- cache(lapplyPartition(textFile(sc, args[[1]]), readPartition)) - -# Initialize w to a random value -w <- runif(n=D, min = -1, max = 1) -cat("Initial w: ", w, "\n") - -# Compute logistic regression gradient for a matrix of data points -gradient <- function(partition) { - partition = partition[[1]] - Y <- partition[, 1] # point labels (first column of input file) - X <- partition[, -1] # point coordinates - - # For each point (x, y), compute gradient function - dot <- X %*% w - logit <- 1 / (1 + exp(-Y * dot)) - grad <- t(X) %*% ((logit - 1) * Y) - list(grad) -} - -for (i in 1:iterations) { - cat("On iteration ", i, "\n") - w <- w - reduce(lapplyPartition(points, gradient), "+") -} - -cat("Final w: ", w, "\n") diff --git a/examples/src/main/r/pi.R b/examples/src/main/r/pi.R deleted file mode 100644 index aa7a833e147a0..0000000000000 --- a/examples/src/main/r/pi.R +++ /dev/null @@ -1,46 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -library(SparkR) - -args <- commandArgs(trailing = TRUE) - -sc <- sparkR.init(appName = "PiR") - -slices <- ifelse(length(args) > 1, as.integer(args[[2]]), 2) - -n <- 100000 * slices - -piFunc <- function(elem) { - rands <- runif(n = 2, min = -1, max = 1) - val <- ifelse((rands[1]^2 + rands[2]^2) < 1, 1.0, 0.0) - val -} - - -piFuncVec <- function(elems) { - message(length(elems)) - rands1 <- runif(n = length(elems), min = -1, max = 1) - rands2 <- runif(n = length(elems), min = -1, max = 1) - val <- ifelse((rands1^2 + rands2^2) < 1, 1.0, 0.0) - sum(val) -} - -rdd <- parallelize(sc, 1:n, slices) -count <- reduce(lapplyPartition(rdd, piFuncVec), sum) -cat("Pi is roughly", 4.0 * count / n, "\n") -cat("Num elements in RDD ", count(rdd), "\n") diff --git a/examples/src/main/r/wordcount.R b/examples/src/main/r/wordcount.R deleted file mode 100644 index b734cb0ecf55b..0000000000000 --- a/examples/src/main/r/wordcount.R +++ /dev/null @@ -1,42 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -library(SparkR) - -args <- commandArgs(trailing = TRUE) - -if (length(args) != 1) { - print("Usage: wordcount ") - q("no") -} - -# Initialize Spark context -sc <- sparkR.init(appName = "RwordCount") -lines <- textFile(sc, args[[1]]) - -words <- flatMap(lines, - function(line) { - strsplit(line, " ")[[1]] - }) -wordCount <- lapply(words, function(word) { list(word, 1L) }) - -counts <- reduceByKey(wordCount, "+", 2L) -output <- collect(counts) - -for (wordcount in output) { - cat(wordcount[[1]], ": ", wordcount[[2]], "\n") -} From 316a5c0423ba3688cacd3acc3c5b5571e8a71d1d Mon Sep 17 00:00:00 2001 From: jerryshao Date: Wed, 6 May 2015 17:44:43 -0700 Subject: [PATCH 19/75] [SPARK-7396] [STREAMING] [EXAMPLE] Update KafkaWordCountProducer to use new Producer API Otherwise it will throw exception: ``` Exception in thread "main" kafka.common.FailedToSendMessageException: Failed to send messages after 3 tries. at kafka.producer.async.DefaultEventHandler.handle(DefaultEventHandler.scala:90) at kafka.producer.Producer.send(Producer.scala:77) at org.apache.spark.examples.streaming.KafkaWordCountProducer$.main(KafkaWordCount.scala:96) at org.apache.spark.examples.streaming.KafkaWordCountProducer.main(KafkaWordCount.scala) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:606) at org.apache.spark.deploy.SparkSubmit$.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:623) at org.apache.spark.deploy.SparkSubmit$.doRunMain$1(SparkSubmit.scala:169) at org.apache.spark.deploy.SparkSubmit$.submit(SparkSubmit.scala:192) at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:111) at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala) ``` Author: jerryshao Closes #5936 from jerryshao/SPARK-7396 and squashes the following commits: 270bbe2 [jerryshao] Fix Kafka Produce throw Exception issue --- .../examples/streaming/KafkaWordCount.scala | 24 ++++++++++--------- 1 file changed, 13 insertions(+), 11 deletions(-) diff --git a/examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala b/examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala index 387c0e421334b..f407367a54f6c 100644 --- a/examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala +++ b/examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala @@ -17,9 +17,9 @@ package org.apache.spark.examples.streaming -import java.util.Properties +import java.util.HashMap -import kafka.producer._ +import org.apache.kafka.clients.producer.{ProducerConfig, KafkaProducer, ProducerRecord} import org.apache.spark.streaming._ import org.apache.spark.streaming.kafka._ @@ -77,23 +77,25 @@ object KafkaWordCountProducer { val Array(brokers, topic, messagesPerSec, wordsPerMessage) = args // Zookeeper connection properties - val props = new Properties() - props.put("metadata.broker.list", brokers) - props.put("serializer.class", "kafka.serializer.StringEncoder") + val props = new HashMap[String, Object]() + props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokers) + props.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, + "org.apache.kafka.common.serialization.StringSerializer") + props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, + "org.apache.kafka.common.serialization.StringSerializer") - val config = new ProducerConfig(props) - val producer = new Producer[String, String](config) + val producer = new KafkaProducer[String, String](props) // Send some messages while(true) { - val messages = (1 to messagesPerSec.toInt).map { messageNum => + (1 to messagesPerSec.toInt).foreach { messageNum => val str = (1 to wordsPerMessage.toInt).map(x => scala.util.Random.nextInt(10).toString) .mkString(" ") - new KeyedMessage[String, String](topic, str) - }.toArray + val message = new ProducerRecord[String, String](topic, null, str) + producer.send(message) + } - producer.send(messages: _*) Thread.sleep(100) } } From 8fa6829f5e6d8b8f33097c3f20fcb96198d4031b Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 6 May 2015 17:52:34 -0700 Subject: [PATCH 20/75] [SPARK-7371] [SPARK-7377] [SPARK-7408] DAG visualization addendum (#5729) This is a follow-up patch for #5729. **[SPARK-7408]** Move as much style code from JS to CSS as possible **[SPARK-7377]** Fix JS error if a job / stage contains only one RDD **[SPARK-7371]** Decrease emphasis on RDD on stage page as requested by mateiz pwendell This patch also includes general code clean up. Author: Andrew Or Closes #5954 from andrewor14/viz-emphasize-rdd and squashes the following commits: 3c0d4f0 [Andrew Or] Guard against JS error by rendering arrows only if needed f23e15b [Andrew Or] Merge branch 'master' of github.com:apache/spark into viz-emphasize-rdd 565801f [Andrew Or] Clean up code 9dab5f0 [Andrew Or] Move styling from JS to CSS + clean up code 107c0b6 [Andrew Or] Tweak background color, stroke width, font size etc. 1610c62 [Andrew Or] Implement cluster padding for stage page --- .../apache/spark/ui/static/dagre-d3.min.js | 14 +- .../apache/spark/ui/static/spark-dag-viz.css | 83 ++++ .../apache/spark/ui/static/spark-dag-viz.js | 357 ++++++++++-------- .../scala/org/apache/spark/ui/UIUtils.scala | 14 +- .../spark/ui/scope/RDDOperationGraph.scala | 19 +- 5 files changed, 310 insertions(+), 177 deletions(-) create mode 100644 core/src/main/resources/org/apache/spark/ui/static/spark-dag-viz.css diff --git a/core/src/main/resources/org/apache/spark/ui/static/dagre-d3.min.js b/core/src/main/resources/org/apache/spark/ui/static/dagre-d3.min.js index 6d2da25024a83..6beb7db855ca0 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/dagre-d3.min.js +++ b/core/src/main/resources/org/apache/spark/ui/static/dagre-d3.min.js @@ -1,4 +1,4 @@ -/*v0.4.3 with 1 additional commit (see http://github.com/andrewor14/dagre-d3)*/(function(f){if(typeof exports==="object"&&typeof module!=="undefined"){module.exports=f()}else if(typeof define==="function"&&define.amd){define([],f)}else{var g;if(typeof window!=="undefined"){g=window}else if(typeof global!=="undefined"){g=global}else if(typeof self!=="undefined"){g=self}else{g=this}g.dagreD3=f()}})(function(){var define,module,exports;return function e(t,n,r){function s(o,u){if(!n[o]){if(!t[o]){var a=typeof require=="function"&&require;if(!u&&a)return a(o,!0);if(i)return i(o,!0);var f=new Error("Cannot find module '"+o+"'");throw f.code="MODULE_NOT_FOUND",f}var l=n[o]={exports:{}};t[o][0].call(l.exports,function(e){var n=t[o][1][e];return s(n?n:e)},l,l.exports,e,t,n,r)}return n[o].exports}var i=typeof require=="function"&&require;for(var o=0;o0}},{}],14:[function(require,module,exports){module.exports=intersectNode;function intersectNode(node,point){return node.intersect(point)}},{}],15:[function(require,module,exports){var intersectLine=require("./intersect-line");module.exports=intersectPolygon;function intersectPolygon(node,polyPoints,point){var x1=node.x;var y1=node.y;var intersections=[];var minX=Number.POSITIVE_INFINITY,minY=Number.POSITIVE_INFINITY;polyPoints.forEach(function(entry){minX=Math.min(minX,entry.x);minY=Math.min(minY,entry.y)});var left=x1-node.width/2-minX;var top=y1-node.height/2-minY;for(var i=0;i1){intersections.sort(function(p,q){var pdx=p.x-point.x,pdy=p.y-point.y,distp=Math.sqrt(pdx*pdx+pdy*pdy),qdx=q.x-point.x,qdy=q.y-point.y,distq=Math.sqrt(qdx*qdx+qdy*qdy);return distpMath.abs(dx)*h){if(dy<0){h=-h}sx=dy===0?0:h*dx/dy;sy=h}else{if(dx<0){w=-w}sx=w;sy=dx===0?0:w*dy/dx}return{x:x+sx,y:y+sy}}},{}],17:[function(require,module,exports){var util=require("../util");module.exports=addHtmlLabel;function addHtmlLabel(root,node){var fo=root.append("foreignObject").attr("width","100000");var div=fo.append("xhtml:div");var label=node.label;switch(typeof label){case"function":div.insert(label);break;case"object":div.insert(function(){return label});break;default:div.html(label)}util.applyStyle(div,node.labelStyle);div.style("display","inline-block");div.style("white-space","nowrap");var w,h;div.each(function(){w=this.clientWidth;h=this.clientHeight});fo.attr("width",w).attr("height",h);return fo}},{"../util":25}],18:[function(require,module,exports){var addTextLabel=require("./add-text-label"),addHtmlLabel=require("./add-html-label");module.exports=addLabel;function addLabel(root,node){var label=node.label;var labelSvg=root.append("g");if(typeof label!=="string"||node.labelType==="html"){addHtmlLabel(labelSvg,node)}else{addTextLabel(labelSvg,node)}var labelBBox=labelSvg.node().getBBox();labelSvg.attr("transform","translate("+-labelBBox.width/2+","+-labelBBox.height/2+")");return labelSvg}},{"./add-html-label":17,"./add-text-label":19}],19:[function(require,module,exports){var util=require("../util");module.exports=addTextLabel;function addTextLabel(root,node){var domNode=root.append("text");var lines=processEscapeSequences(node.label).split("\n");for(var i=0;i0;--i){entry=buckets[i].dequeue();if(entry){results=results.concat(removeNode(g,buckets,zeroIdx,entry,true));break}}}}return results}function removeNode(g,buckets,zeroIdx,entry,collectPredecessors){var results=collectPredecessors?[]:undefined;_.each(g.inEdges(entry.v),function(edge){var weight=g.edge(edge),uEntry=g.node(edge.v);if(collectPredecessors){results.push({v:edge.v,w:edge.w})}uEntry.out-=weight;assignBucket(buckets,zeroIdx,uEntry)});_.each(g.outEdges(entry.v),function(edge){var weight=g.edge(edge),w=edge.w,wEntry=g.node(w);wEntry["in"]-=weight;assignBucket(buckets,zeroIdx,wEntry)});g.removeNode(entry.v);return results}function buildState(g,weightFn){var fasGraph=new Graph,maxIn=0,maxOut=0;_.each(g.nodes(),function(v){fasGraph.setNode(v,{v:v,"in":0,out:0})});_.each(g.edges(),function(e){var prevWeight=fasGraph.edge(e.v,e.w)||0,weight=weightFn(e),edgeWeight=prevWeight+weight;fasGraph.setEdge(e.v,e.w,edgeWeight);maxOut=Math.max(maxOut,fasGraph.node(e.v).out+=weight);maxIn=Math.max(maxIn,fasGraph.node(e.w)["in"]+=weight)});var buckets=_.range(maxOut+maxIn+3).map(function(){return new List});var zeroIdx=maxIn+1;_.each(fasGraph.nodes(),function(v){assignBucket(buckets,zeroIdx,fasGraph.node(v))});return{graph:fasGraph,buckets:buckets,zeroIdx:zeroIdx}}function assignBucket(buckets,zeroIdx,entry){if(!entry.out){buckets[0].enqueue(entry)}else if(!entry["in"]){buckets[buckets.length-1].enqueue(entry)}else{buckets[entry.out-entry["in"]+zeroIdx].enqueue(entry)}}},{"./data/list":31,"./graphlib":33,"./lodash":36}],35:[function(require,module,exports){"use strict";var _=require("./lodash"),acyclic=require("./acyclic"),normalize=require("./normalize"),rank=require("./rank"),normalizeRanks=require("./util").normalizeRanks,parentDummyChains=require("./parent-dummy-chains"),removeEmptyRanks=require("./util").removeEmptyRanks,nestingGraph=require("./nesting-graph"),addBorderSegments=require("./add-border-segments"),coordinateSystem=require("./coordinate-system"),order=require("./order"),position=require("./position"),util=require("./util"),Graph=require("./graphlib").Graph;module.exports=layout;function layout(g,opts){var time=opts&&opts.debugTiming?util.time:util.notime;time("layout",function(){var layoutGraph=time(" buildLayoutGraph",function(){return buildLayoutGraph(g)});time(" runLayout",function(){runLayout(layoutGraph,time)});time(" updateInputGraph",function(){updateInputGraph(g,layoutGraph)})})}function runLayout(g,time){time(" makeSpaceForEdgeLabels",function(){makeSpaceForEdgeLabels(g)});time(" removeSelfEdges",function(){removeSelfEdges(g)});time(" acyclic",function(){acyclic.run(g)});time(" nestingGraph.run",function(){nestingGraph.run(g)});time(" rank",function(){rank(util.asNonCompoundGraph(g))});time(" injectEdgeLabelProxies",function(){injectEdgeLabelProxies(g)});time(" removeEmptyRanks",function(){removeEmptyRanks(g)});time(" nestingGraph.cleanup",function(){nestingGraph.cleanup(g)});time(" normalizeRanks",function(){normalizeRanks(g)});time(" assignRankMinMax",function(){assignRankMinMax(g)});time(" removeEdgeLabelProxies",function(){removeEdgeLabelProxies(g)});time(" normalize.run",function(){normalize.run(g)});time(" parentDummyChains",function(){parentDummyChains(g)});time(" addBorderSegments",function(){addBorderSegments(g)});time(" order",function(){order(g)});time(" insertSelfEdges",function(){insertSelfEdges(g)});time(" adjustCoordinateSystem",function(){coordinateSystem.adjust(g)});time(" position",function(){position(g)});time(" positionSelfEdges",function(){positionSelfEdges(g)});time(" removeBorderNodes",function(){removeBorderNodes(g)});time(" normalize.undo",function(){normalize.undo(g)});time(" fixupEdgeLabelCoords",function(){fixupEdgeLabelCoords(g)});time(" undoCoordinateSystem",function(){coordinateSystem.undo(g)});time(" translateGraph",function(){translateGraph(g)});time(" assignNodeIntersects",function(){assignNodeIntersects(g)});time(" reversePoints",function(){reversePointsForReversedEdges(g)});time(" acyclic.undo",function(){acyclic.undo(g)})}function updateInputGraph(inputGraph,layoutGraph){_.each(inputGraph.nodes(),function(v){var inputLabel=inputGraph.node(v),layoutLabel=layoutGraph.node(v);if(inputLabel){inputLabel.x=layoutLabel.x;inputLabel.y=layoutLabel.y;if(layoutGraph.children(v).length){inputLabel.width=layoutLabel.width;inputLabel.height=layoutLabel.height}}});_.each(inputGraph.edges(),function(e){var inputLabel=inputGraph.edge(e),layoutLabel=layoutGraph.edge(e);inputLabel.points=layoutLabel.points;if(_.has(layoutLabel,"x")){inputLabel.x=layoutLabel.x;inputLabel.y=layoutLabel.y}});inputGraph.graph().width=layoutGraph.graph().width;inputGraph.graph().height=layoutGraph.graph().height}var graphNumAttrs=["nodesep","edgesep","ranksep","marginx","marginy"],graphDefaults={ranksep:50,edgesep:20,nodesep:50,rankdir:"tb"},graphAttrs=["acyclicer","ranker","rankdir","align"],nodeNumAttrs=["width","height"],nodeDefaults={width:0,height:0},edgeNumAttrs=["minlen","weight","width","height","labeloffset"],edgeDefaults={minlen:1,weight:1,width:0,height:0,labeloffset:10,labelpos:"r"},edgeAttrs=["labelpos"];function buildLayoutGraph(inputGraph){var g=new Graph({multigraph:true,compound:true}),graph=canonicalize(inputGraph.graph());g.setGraph(_.merge({},graphDefaults,selectNumberAttrs(graph,graphNumAttrs),_.pick(graph,graphAttrs)));_.each(inputGraph.nodes(),function(v){var node=canonicalize(inputGraph.node(v));g.setNode(v,_.defaults(selectNumberAttrs(node,nodeNumAttrs),nodeDefaults));g.setParent(v,inputGraph.parent(v))});_.each(inputGraph.edges(),function(e){var edge=canonicalize(inputGraph.edge(e));g.setEdge(e,_.merge({},edgeDefaults,selectNumberAttrs(edge,edgeNumAttrs),_.pick(edge,edgeAttrs)))});return g}function makeSpaceForEdgeLabels(g){var graph=g.graph(); +module.exports={graphlib:require("./lib/graphlib"),dagre:require("./lib/dagre"),intersect:require("./lib/intersect"),render:require("./lib/render"),util:require("./lib/util"),version:require("./lib/version")}},{"./lib/dagre":8,"./lib/graphlib":9,"./lib/intersect":10,"./lib/render":23,"./lib/util":25,"./lib/version":26}],2:[function(require,module,exports){var util=require("./util");module.exports={"default":normal,normal:normal,vee:vee,undirected:undirected};function normal(parent,id,edge,type){var marker=parent.append("marker").attr("id",id).attr("viewBox","0 0 10 10").attr("refX",9).attr("refY",5).attr("markerUnits","strokeWidth").attr("markerWidth",8).attr("markerHeight",6).attr("orient","auto");var path=marker.append("path").attr("d","M 0 0 L 10 5 L 0 10 z").style("stroke-width",1).style("stroke-dasharray","1,0");util.applyStyle(path,edge[type+"Style"])}function vee(parent,id,edge,type){var marker=parent.append("marker").attr("id",id).attr("viewBox","0 0 10 10").attr("refX",9).attr("refY",5).attr("markerUnits","strokeWidth").attr("markerWidth",8).attr("markerHeight",6).attr("orient","auto");var path=marker.append("path").attr("d","M 0 0 L 10 5 L 0 10 L 4 5 z").style("stroke-width",1).style("stroke-dasharray","1,0");util.applyStyle(path,edge[type+"Style"])}function undirected(parent,id,edge,type){var marker=parent.append("marker").attr("id",id).attr("viewBox","0 0 10 10").attr("refX",9).attr("refY",5).attr("markerUnits","strokeWidth").attr("markerWidth",8).attr("markerHeight",6).attr("orient","auto");var path=marker.append("path").attr("d","M 0 5 L 10 5").style("stroke-width",1).style("stroke-dasharray","1,0");util.applyStyle(path,edge[type+"Style"])}},{"./util":25}],3:[function(require,module,exports){var util=require("./util");module.exports=createClusters;function createClusters(selection,g){var clusters=g.nodes().filter(function(v){return util.isSubgraph(g,v)}),svgClusters=selection.selectAll("g.cluster").data(clusters,function(v){return v});svgClusters.enter().append("g").attr("id",function(v){return"cluster_"+v.replace(/^cluster/,"")}).attr("name",function(v){return g.node(v).label}).attr("class","cluster").style("opacity",0).append("rect");util.applyTransition(svgClusters.exit(),g).style("opacity",0).remove();util.applyTransition(svgClusters,g).style("opacity",1);util.applyTransition(svgClusters.selectAll("rect"),g).attr("width",function(v){var node=g.node(v);return node.width+node.paddingLeft+node.paddingRight}).attr("height",function(v){var node=g.node(v);return node.height+node.paddingTop+node.paddingBottom}).attr("x",function(v){var node=g.node(v);return node.x-node.width/2-node.paddingLeft}).attr("y",function(v){var node=g.node(v);return node.y-node.height/2-node.paddingTop})}},{"./util":25}],4:[function(require,module,exports){"use strict";var _=require("./lodash"),addLabel=require("./label/add-label"),util=require("./util"),d3=require("./d3");module.exports=createEdgeLabels;function createEdgeLabels(selection,g){var svgEdgeLabels=selection.selectAll("g.edgeLabel").data(g.edges(),function(e){return util.edgeToId(e)}).classed("update",true);svgEdgeLabels.selectAll("*").remove();svgEdgeLabels.enter().append("g").classed("edgeLabel",true).style("opacity",0);svgEdgeLabels.each(function(e){var edge=g.edge(e),label=addLabel(d3.select(this),g.edge(e),0,0).classed("label",true),bbox=label.node().getBBox();if(edge.labelId){label.attr("id",edge.labelId)}if(!_.has(edge,"width")){edge.width=bbox.width}if(!_.has(edge,"height")){edge.height=bbox.height}});util.applyTransition(svgEdgeLabels.exit(),g).style("opacity",0).remove();return svgEdgeLabels}},{"./d3":7,"./label/add-label":18,"./lodash":20,"./util":25}],5:[function(require,module,exports){"use strict";var _=require("./lodash"),intersectNode=require("./intersect/intersect-node"),util=require("./util"),d3=require("./d3");module.exports=createEdgePaths;function createEdgePaths(selection,g,arrows){var svgPaths=selection.selectAll("g.edgePath").data(g.edges(),function(e){return util.edgeToId(e)}).classed("update",true);enter(svgPaths,g);exit(svgPaths,g);util.applyTransition(svgPaths,g).style("opacity",1);svgPaths.each(function(e){var domEdge=d3.select(this);var edge=g.edge(e);edge.elem=this;if(edge.id){domEdge.attr("id",edge.id)}util.applyClass(domEdge,edge["class"],(domEdge.classed("update")?"update ":"")+"edgePath")});svgPaths.selectAll("path.path").each(function(e){var edge=g.edge(e);edge.arrowheadId=_.uniqueId("arrowhead");var domEdge=d3.select(this).attr("marker-end",function(){return"url(#"+edge.arrowheadId+")"}).style("fill","none");util.applyTransition(domEdge,g).attr("d",function(e){return calcPoints(g,e)});util.applyStyle(domEdge,edge.style)});svgPaths.selectAll("defs *").remove();svgPaths.selectAll("defs").each(function(e){var edge=g.edge(e),arrowhead=arrows[edge.arrowhead];arrowhead(d3.select(this),edge.arrowheadId,edge,"arrowhead")});return svgPaths}function calcPoints(g,e){var edge=g.edge(e),tail=g.node(e.v),head=g.node(e.w),points=edge.points.slice(1,edge.points.length-1);points.unshift(intersectNode(tail,points[0]));points.push(intersectNode(head,points[points.length-1]));return createLine(edge,points)}function createLine(edge,points){var line=d3.svg.line().x(function(d){return d.x}).y(function(d){return d.y});if(_.has(edge,"lineInterpolate")){line.interpolate(edge.lineInterpolate)}if(_.has(edge,"lineTension")){line.tension(Number(edge.lineTension))}return line(points)}function getCoords(elem){var bbox=elem.getBBox(),matrix=elem.getTransformToElement(elem.ownerSVGElement).translate(bbox.width/2,bbox.height/2);return{x:matrix.e,y:matrix.f}}function enter(svgPaths,g){var svgPathsEnter=svgPaths.enter().append("g").attr("class","edgePath").style("opacity",0);svgPathsEnter.append("path").attr("class","path").attr("d",function(e){var edge=g.edge(e),sourceElem=g.node(e.v).elem,points=_.range(edge.points.length).map(function(){return getCoords(sourceElem)});return createLine(edge,points)});svgPathsEnter.append("defs")}function exit(svgPaths,g){var svgPathExit=svgPaths.exit();util.applyTransition(svgPathExit,g).style("opacity",0).remove();util.applyTransition(svgPathExit.select("path.path"),g).attr("d",function(e){var source=g.node(e.v);if(source){var points=_.range(this.pathSegList.length).map(function(){return source});return createLine({},points)}else{return d3.select(this).attr("d")}})}},{"./d3":7,"./intersect/intersect-node":14,"./lodash":20,"./util":25}],6:[function(require,module,exports){"use strict";var _=require("./lodash"),addLabel=require("./label/add-label"),util=require("./util"),d3=require("./d3");module.exports=createNodes;function createNodes(selection,g,shapes){var simpleNodes=g.nodes().filter(function(v){return!util.isSubgraph(g,v)});var svgNodes=selection.selectAll("g.node").data(simpleNodes,function(v){return v}).classed("update",true);svgNodes.selectAll("*").remove();svgNodes.enter().append("g").attr("id",function(v){return"node_"+v}).attr("name",function(v){return g.node(v).label}).attr("class","node").style("opacity",0);svgNodes.each(function(v){var node=g.node(v),thisGroup=d3.select(this),labelGroup=thisGroup.append("g").attr("class","label"),labelDom=addLabel(labelGroup,node),shape=shapes[node.shape],bbox=_.pick(labelDom.node().getBBox(),"width","height");node.elem=this;if(node.id){thisGroup.attr("id",node.id)}if(node.labelId){labelGroup.attr("id",node.labelId)}util.applyClass(thisGroup,node["class"],(thisGroup.classed("update")?"update ":"")+"node");if(_.has(node,"width")){bbox.width=node.width}if(_.has(node,"height")){bbox.height=node.height}bbox.width+=node.paddingLeft+node.paddingRight;bbox.height+=node.paddingTop+node.paddingBottom;labelGroup.attr("transform","translate("+(node.paddingLeft-node.paddingRight)/2+","+(node.paddingTop-node.paddingBottom)/2+")");var shapeSvg=shape(d3.select(this),bbox,node);util.applyStyle(shapeSvg,node.style);var shapeBBox=shapeSvg.node().getBBox();node.width=shapeBBox.width;node.height=shapeBBox.height});util.applyTransition(svgNodes.exit(),g).style("opacity",0).remove();return svgNodes}},{"./d3":7,"./label/add-label":18,"./lodash":20,"./util":25}],7:[function(require,module,exports){module.exports=window.d3},{}],8:[function(require,module,exports){var dagre;if(require){try{dagre=require("dagre")}catch(e){}}if(!dagre){dagre=window.dagre}module.exports=dagre},{dagre:27}],9:[function(require,module,exports){var graphlib;if(require){try{graphlib=require("graphlib")}catch(e){}}if(!graphlib){graphlib=window.graphlib}module.exports=graphlib},{graphlib:57}],10:[function(require,module,exports){module.exports={node:require("./intersect-node"),circle:require("./intersect-circle"),ellipse:require("./intersect-ellipse"),polygon:require("./intersect-polygon"),rect:require("./intersect-rect")}},{"./intersect-circle":11,"./intersect-ellipse":12,"./intersect-node":14,"./intersect-polygon":15,"./intersect-rect":16}],11:[function(require,module,exports){var intersectEllipse=require("./intersect-ellipse");module.exports=intersectCircle;function intersectCircle(node,rx,point){return intersectEllipse(node,rx,rx,point)}},{"./intersect-ellipse":12}],12:[function(require,module,exports){module.exports=intersectEllipse;function intersectEllipse(node,rx,ry,point){var cx=node.x;var cy=node.y;var px=cx-point.x;var py=cy-point.y;var det=Math.sqrt(rx*rx*py*py+ry*ry*px*px);var dx=Math.abs(rx*ry*px/det);if(point.x0}},{}],14:[function(require,module,exports){module.exports=intersectNode;function intersectNode(node,point){return node.intersect(point)}},{}],15:[function(require,module,exports){var intersectLine=require("./intersect-line");module.exports=intersectPolygon;function intersectPolygon(node,polyPoints,point){var x1=node.x;var y1=node.y;var intersections=[];var minX=Number.POSITIVE_INFINITY,minY=Number.POSITIVE_INFINITY;polyPoints.forEach(function(entry){minX=Math.min(minX,entry.x);minY=Math.min(minY,entry.y)});var left=x1-node.width/2-minX;var top=y1-node.height/2-minY;for(var i=0;i1){intersections.sort(function(p,q){var pdx=p.x-point.x,pdy=p.y-point.y,distp=Math.sqrt(pdx*pdx+pdy*pdy),qdx=q.x-point.x,qdy=q.y-point.y,distq=Math.sqrt(qdx*qdx+qdy*qdy);return distpMath.abs(dx)*h){if(dy<0){h=-h}sx=dy===0?0:h*dx/dy;sy=h}else{if(dx<0){w=-w}sx=w;sy=dx===0?0:w*dy/dx}return{x:x+sx,y:y+sy}}},{}],17:[function(require,module,exports){var util=require("../util");module.exports=addHtmlLabel;function addHtmlLabel(root,node){var fo=root.append("foreignObject").attr("width","100000");var div=fo.append("xhtml:div");var label=node.label;switch(typeof label){case"function":div.insert(label);break;case"object":div.insert(function(){return label});break;default:div.html(label)}util.applyStyle(div,node.labelStyle);div.style("display","inline-block");div.style("white-space","nowrap");var w,h;div.each(function(){w=this.clientWidth;h=this.clientHeight});fo.attr("width",w).attr("height",h);return fo}},{"../util":25}],18:[function(require,module,exports){var addTextLabel=require("./add-text-label"),addHtmlLabel=require("./add-html-label");module.exports=addLabel;function addLabel(root,node){var label=node.label;var labelSvg=root.append("g");if(typeof label!=="string"||node.labelType==="html"){addHtmlLabel(labelSvg,node)}else{addTextLabel(labelSvg,node)}var labelBBox=labelSvg.node().getBBox();labelSvg.attr("transform","translate("+-labelBBox.width/2+","+-labelBBox.height/2+")");return labelSvg}},{"./add-html-label":17,"./add-text-label":19}],19:[function(require,module,exports){var util=require("../util");module.exports=addTextLabel;function addTextLabel(root,node){var domNode=root.append("text");var lines=processEscapeSequences(node.label).split("\n");for(var i=0;i0;--i){entry=buckets[i].dequeue();if(entry){results=results.concat(removeNode(g,buckets,zeroIdx,entry,true));break}}}}return results}function removeNode(g,buckets,zeroIdx,entry,collectPredecessors){var results=collectPredecessors?[]:undefined;_.each(g.inEdges(entry.v),function(edge){var weight=g.edge(edge),uEntry=g.node(edge.v);if(collectPredecessors){results.push({v:edge.v,w:edge.w})}uEntry.out-=weight;assignBucket(buckets,zeroIdx,uEntry)});_.each(g.outEdges(entry.v),function(edge){var weight=g.edge(edge),w=edge.w,wEntry=g.node(w);wEntry["in"]-=weight;assignBucket(buckets,zeroIdx,wEntry)});g.removeNode(entry.v);return results}function buildState(g,weightFn){var fasGraph=new Graph,maxIn=0,maxOut=0;_.each(g.nodes(),function(v){fasGraph.setNode(v,{v:v,"in":0,out:0})});_.each(g.edges(),function(e){var prevWeight=fasGraph.edge(e.v,e.w)||0,weight=weightFn(e),edgeWeight=prevWeight+weight;fasGraph.setEdge(e.v,e.w,edgeWeight);maxOut=Math.max(maxOut,fasGraph.node(e.v).out+=weight);maxIn=Math.max(maxIn,fasGraph.node(e.w)["in"]+=weight)});var buckets=_.range(maxOut+maxIn+3).map(function(){return new List});var zeroIdx=maxIn+1;_.each(fasGraph.nodes(),function(v){assignBucket(buckets,zeroIdx,fasGraph.node(v))});return{graph:fasGraph,buckets:buckets,zeroIdx:zeroIdx}}function assignBucket(buckets,zeroIdx,entry){if(!entry.out){buckets[0].enqueue(entry)}else if(!entry["in"]){buckets[buckets.length-1].enqueue(entry)}else{buckets[entry.out-entry["in"]+zeroIdx].enqueue(entry)}}},{"./data/list":31,"./graphlib":33,"./lodash":36}],35:[function(require,module,exports){"use strict";var _=require("./lodash"),acyclic=require("./acyclic"),normalize=require("./normalize"),rank=require("./rank"),normalizeRanks=require("./util").normalizeRanks,parentDummyChains=require("./parent-dummy-chains"),removeEmptyRanks=require("./util").removeEmptyRanks,nestingGraph=require("./nesting-graph"),addBorderSegments=require("./add-border-segments"),coordinateSystem=require("./coordinate-system"),order=require("./order"),position=require("./position"),util=require("./util"),Graph=require("./graphlib").Graph;module.exports=layout;function layout(g,opts){var time=opts&&opts.debugTiming?util.time:util.notime;time("layout",function(){var layoutGraph=time(" buildLayoutGraph",function(){return buildLayoutGraph(g)});time(" runLayout",function(){runLayout(layoutGraph,time)});time(" updateInputGraph",function(){updateInputGraph(g,layoutGraph)})})}function runLayout(g,time){time(" makeSpaceForEdgeLabels",function(){makeSpaceForEdgeLabels(g)});time(" removeSelfEdges",function(){removeSelfEdges(g)});time(" acyclic",function(){acyclic.run(g)});time(" nestingGraph.run",function(){nestingGraph.run(g)});time(" rank",function(){rank(util.asNonCompoundGraph(g))});time(" injectEdgeLabelProxies",function(){injectEdgeLabelProxies(g)});time(" removeEmptyRanks",function(){removeEmptyRanks(g)});time(" nestingGraph.cleanup",function(){nestingGraph.cleanup(g)});time(" normalizeRanks",function(){normalizeRanks(g)});time(" assignRankMinMax",function(){assignRankMinMax(g)});time(" removeEdgeLabelProxies",function(){removeEdgeLabelProxies(g)});time(" normalize.run",function(){normalize.run(g)});time(" parentDummyChains",function(){parentDummyChains(g)});time(" addBorderSegments",function(){addBorderSegments(g)});time(" order",function(){order(g)});time(" insertSelfEdges",function(){insertSelfEdges(g)});time(" adjustCoordinateSystem",function(){coordinateSystem.adjust(g)});time(" position",function(){position(g)});time(" positionSelfEdges",function(){positionSelfEdges(g)});time(" removeBorderNodes",function(){removeBorderNodes(g)});time(" normalize.undo",function(){normalize.undo(g)});time(" fixupEdgeLabelCoords",function(){fixupEdgeLabelCoords(g)});time(" undoCoordinateSystem",function(){coordinateSystem.undo(g)});time(" translateGraph",function(){translateGraph(g)});time(" assignNodeIntersects",function(){assignNodeIntersects(g)});time(" reversePoints",function(){reversePointsForReversedEdges(g)});time(" acyclic.undo",function(){acyclic.undo(g)})}function updateInputGraph(inputGraph,layoutGraph){_.each(inputGraph.nodes(),function(v){var inputLabel=inputGraph.node(v),layoutLabel=layoutGraph.node(v);if(inputLabel){inputLabel.x=layoutLabel.x;inputLabel.y=layoutLabel.y;if(layoutGraph.children(v).length){inputLabel.width=layoutLabel.width;inputLabel.height=layoutLabel.height}}});_.each(inputGraph.edges(),function(e){var inputLabel=inputGraph.edge(e),layoutLabel=layoutGraph.edge(e);inputLabel.points=layoutLabel.points;if(_.has(layoutLabel,"x")){inputLabel.x=layoutLabel.x;inputLabel.y=layoutLabel.y}});inputGraph.graph().width=layoutGraph.graph().width;inputGraph.graph().height=layoutGraph.graph().height}var graphNumAttrs=["nodesep","edgesep","ranksep","marginx","marginy"],graphDefaults={ranksep:50,edgesep:20,nodesep:50,rankdir:"tb"},graphAttrs=["acyclicer","ranker","rankdir","align"],nodeNumAttrs=["width","height"],nodeDefaults={width:0,height:0},edgeNumAttrs=["minlen","weight","width","height","labeloffset"],edgeDefaults={minlen:1,weight:1,width:0,height:0,labeloffset:10,labelpos:"r"},edgeAttrs=["labelpos"];function buildLayoutGraph(inputGraph){var g=new Graph({multigraph:true,compound:true}),graph=canonicalize(inputGraph.graph());g.setGraph(_.merge({},graphDefaults,selectNumberAttrs(graph,graphNumAttrs),_.pick(graph,graphAttrs))); -graph.ranksep/=2;_.each(g.edges(),function(e){var edge=g.edge(e);edge.minlen*=2;if(edge.labelpos.toLowerCase()!=="c"){if(graph.rankdir==="TB"||graph.rankdir==="BT"){edge.width+=edge.labeloffset}else{edge.height+=edge.labeloffset}}})}function injectEdgeLabelProxies(g){_.each(g.edges(),function(e){var edge=g.edge(e);if(edge.width&&edge.height){var v=g.node(e.v),w=g.node(e.w),label={rank:(w.rank-v.rank)/2+v.rank,e:e};util.addDummyNode(g,"edge-proxy",label,"_ep")}})}function assignRankMinMax(g){var maxRank=0;_.each(g.nodes(),function(v){var node=g.node(v);if(node.borderTop){node.minRank=g.node(node.borderTop).rank;node.maxRank=g.node(node.borderBottom).rank;maxRank=_.max(maxRank,node.maxRank)}});g.graph().maxRank=maxRank}function removeEdgeLabelProxies(g){_.each(g.nodes(),function(v){var node=g.node(v);if(node.dummy==="edge-proxy"){g.edge(node.e).labelRank=node.rank;g.removeNode(v)}})}function translateGraph(g){var minX=Number.POSITIVE_INFINITY,maxX=0,minY=Number.POSITIVE_INFINITY,maxY=0,graphLabel=g.graph(),marginX=graphLabel.marginx||0,marginY=graphLabel.marginy||0;function getExtremes(attrs){var x=attrs.x,y=attrs.y,w=attrs.width,h=attrs.height;minX=Math.min(minX,x-w/2);maxX=Math.max(maxX,x+w/2);minY=Math.min(minY,y-h/2);maxY=Math.max(maxY,y+h/2)}_.each(g.nodes(),function(v){getExtremes(g.node(v))});_.each(g.edges(),function(e){var edge=g.edge(e);if(_.has(edge,"x")){getExtremes(edge)}});minX-=marginX;minY-=marginY;_.each(g.nodes(),function(v){var node=g.node(v);node.x-=minX;node.y-=minY});_.each(g.edges(),function(e){var edge=g.edge(e);_.each(edge.points,function(p){p.x-=minX;p.y-=minY});if(_.has(edge,"x")){edge.x-=minX}if(_.has(edge,"y")){edge.y-=minY}});graphLabel.width=maxX-minX+marginX;graphLabel.height=maxY-minY+marginY}function assignNodeIntersects(g){_.each(g.edges(),function(e){var edge=g.edge(e),nodeV=g.node(e.v),nodeW=g.node(e.w),p1,p2;if(!edge.points){edge.points=[];p1=nodeW;p2=nodeV}else{p1=edge.points[0];p2=edge.points[edge.points.length-1]}edge.points.unshift(util.intersectRect(nodeV,p1));edge.points.push(util.intersectRect(nodeW,p2))})}function fixupEdgeLabelCoords(g){_.each(g.edges(),function(e){var edge=g.edge(e);if(_.has(edge,"x")){if(edge.labelpos==="l"||edge.labelpos==="r"){edge.width-=edge.labeloffset}switch(edge.labelpos){case"l":edge.x-=edge.width/2+edge.labeloffset;break;case"r":edge.x+=edge.width/2+edge.labeloffset;break}}})}function reversePointsForReversedEdges(g){_.each(g.edges(),function(e){var edge=g.edge(e);if(edge.reversed){edge.points.reverse()}})}function removeBorderNodes(g){_.each(g.nodes(),function(v){if(g.children(v).length){var node=g.node(v),t=g.node(node.borderTop),b=g.node(node.borderBottom),l=g.node(_.last(node.borderLeft)),r=g.node(_.last(node.borderRight));node.width=Math.abs(r.x-l.x);node.height=Math.abs(b.y-t.y);node.x=l.x+node.width/2;node.y=t.y+node.height/2}});_.each(g.nodes(),function(v){if(g.node(v).dummy==="border"){g.removeNode(v)}})}function removeSelfEdges(g){_.each(g.edges(),function(e){if(e.v===e.w){var node=g.node(e.v);if(!node.selfEdges){node.selfEdges=[]}node.selfEdges.push({e:e,label:g.edge(e)});g.removeEdge(e)}})}function insertSelfEdges(g){var layers=util.buildLayerMatrix(g);_.each(layers,function(layer){var orderShift=0;_.each(layer,function(v,i){var node=g.node(v);node.order=i+orderShift;_.each(node.selfEdges,function(selfEdge){util.addDummyNode(g,"selfedge",{width:selfEdge.label.width,height:selfEdge.label.height,rank:node.rank,order:i+ ++orderShift,e:selfEdge.e,label:selfEdge.label},"_se")});delete node.selfEdges})})}function positionSelfEdges(g){_.each(g.nodes(),function(v){var node=g.node(v);if(node.dummy==="selfedge"){var selfNode=g.node(node.e.v),x=selfNode.x+selfNode.width/2,y=selfNode.y,dx=node.x-x,dy=selfNode.height/2;g.setEdge(node.e,node.label);g.removeNode(v);node.label.points=[{x:x+2*dx/3,y:y-dy},{x:x+5*dx/6,y:y-dy},{x:x+dx,y:y},{x:x+5*dx/6,y:y+dy},{x:x+2*dx/3,y:y+dy}];node.label.x=node.x;node.label.y=node.y}})}function selectNumberAttrs(obj,attrs){return _.mapValues(_.pick(obj,attrs),Number)}function canonicalize(attrs){var newAttrs={};_.each(attrs,function(v,k){newAttrs[k.toLowerCase()]=v});return newAttrs}},{"./acyclic":28,"./add-border-segments":29,"./coordinate-system":30,"./graphlib":33,"./lodash":36,"./nesting-graph":37,"./normalize":38,"./order":43,"./parent-dummy-chains":48,"./position":50,"./rank":52,"./util":55}],36:[function(require,module,exports){arguments[4][20][0].apply(exports,arguments)},{dup:20,lodash:77}],37:[function(require,module,exports){var _=require("./lodash"),util=require("./util");module.exports={run:run,cleanup:cleanup};function run(g){var root=util.addDummyNode(g,"root",{},"_root"),depths=treeDepths(g),height=_.max(depths)-1,nodeSep=2*height+1;g.graph().nestingRoot=root;_.each(g.edges(),function(e){g.edge(e).minlen*=nodeSep});var weight=sumWeights(g)+1;_.each(g.children(),function(child){dfs(g,root,nodeSep,weight,height,depths,child)});g.graph().nodeRankFactor=nodeSep}function dfs(g,root,nodeSep,weight,height,depths,v){var children=g.children(v);if(!children.length){if(v!==root){g.setEdge(root,v,{weight:0,minlen:nodeSep})}return}var top=util.addBorderNode(g,"_bt"),bottom=util.addBorderNode(g,"_bb"),label=g.node(v);g.setParent(top,v);label.borderTop=top;g.setParent(bottom,v);label.borderBottom=bottom;_.each(children,function(child){dfs(g,root,nodeSep,weight,height,depths,child);var childNode=g.node(child),childTop=childNode.borderTop?childNode.borderTop:child,childBottom=childNode.borderBottom?childNode.borderBottom:child,thisWeight=childNode.borderTop?weight:2*weight,minlen=childTop!==childBottom?1:height-depths[v]+1;g.setEdge(top,childTop,{weight:thisWeight,minlen:minlen,nestingEdge:true});g.setEdge(childBottom,bottom,{weight:thisWeight,minlen:minlen,nestingEdge:true})});if(!g.parent(v)){g.setEdge(root,top,{weight:0,minlen:height+depths[v]})}}function treeDepths(g){var depths={};function dfs(v,depth){var children=g.children(v);if(children&&children.length){_.each(children,function(child){dfs(child,depth+1)})}depths[v]=depth}_.each(g.children(),function(v){dfs(v,1)});return depths}function sumWeights(g){return _.reduce(g.edges(),function(acc,e){return acc+g.edge(e).weight},0)}function cleanup(g){var graphLabel=g.graph();g.removeNode(graphLabel.nestingRoot);delete graphLabel.nestingRoot;_.each(g.edges(),function(e){var edge=g.edge(e);if(edge.nestingEdge){g.removeEdge(e)}})}},{"./lodash":36,"./util":55}],38:[function(require,module,exports){"use strict";var _=require("./lodash"),util=require("./util");module.exports={run:run,undo:undo};function run(g){g.graph().dummyChains=[];_.each(g.edges(),function(edge){normalizeEdge(g,edge)})}function normalizeEdge(g,e){var v=e.v,vRank=g.node(v).rank,w=e.w,wRank=g.node(w).rank,name=e.name,edgeLabel=g.edge(e),labelRank=edgeLabel.labelRank;if(wRank===vRank+1)return;g.removeEdge(e);var dummy,attrs,i;for(i=0,++vRank;vRank0){if(index%2){weightSum+=tree[index+1]}index=index-1>>1;tree[index]+=entry.weight}cc+=entry.weight*weightSum}));return cc}},{"../lodash":36}],43:[function(require,module,exports){"use strict";var _=require("../lodash"),initOrder=require("./init-order"),crossCount=require("./cross-count"),sortSubgraph=require("./sort-subgraph"),buildLayerGraph=require("./build-layer-graph"),addSubgraphConstraints=require("./add-subgraph-constraints"),Graph=require("../graphlib").Graph,util=require("../util");module.exports=order;function order(g){var maxRank=util.maxRank(g),downLayerGraphs=buildLayerGraphs(g,_.range(1,maxRank+1),"inEdges"),upLayerGraphs=buildLayerGraphs(g,_.range(maxRank-1,-1,-1),"outEdges");var layering=initOrder(g);assignOrder(g,layering);var bestCC=Number.POSITIVE_INFINITY,best;for(var i=0,lastBest=0;lastBest<4;++i,++lastBest){sweepLayerGraphs(i%2?downLayerGraphs:upLayerGraphs,i%4>=2);layering=util.buildLayerMatrix(g);var cc=crossCount(g,layering);if(cc=vEntry.barycenter){mergeEntries(vEntry,uEntry)}}}function handleOut(vEntry){return function(wEntry){wEntry["in"].push(vEntry);if(--wEntry.indegree===0){sourceSet.push(wEntry)}}}while(sourceSet.length){var entry=sourceSet.pop();entries.push(entry);_.each(entry["in"].reverse(),handleIn(entry));_.each(entry.out,handleOut(entry))}return _.chain(entries).filter(function(entry){return!entry.merged}).map(function(entry){return _.pick(entry,["vs","i","barycenter","weight"])}).value()}function mergeEntries(target,source){var sum=0,weight=0;if(target.weight){sum+=target.barycenter*target.weight;weight+=target.weight}if(source.weight){sum+=source.barycenter*source.weight;weight+=source.weight}target.vs=source.vs.concat(target.vs);target.barycenter=sum/weight;target.weight=weight;target.i=Math.min(source.i,target.i);source.merged=true}},{"../lodash":36}],46:[function(require,module,exports){var _=require("../lodash"),barycenter=require("./barycenter"),resolveConflicts=require("./resolve-conflicts"),sort=require("./sort");module.exports=sortSubgraph;function sortSubgraph(g,v,cg,biasRight){var movable=g.children(v),node=g.node(v),bl=node?node.borderLeft:undefined,br=node?node.borderRight:undefined,subgraphs={};if(bl){movable=_.filter(movable,function(w){return w!==bl&&w!==br})}var barycenters=barycenter(g,movable);_.each(barycenters,function(entry){if(g.children(entry.v).length){var subgraphResult=sortSubgraph(g,entry.v,cg,biasRight);subgraphs[entry.v]=subgraphResult;if(_.has(subgraphResult,"barycenter")){mergeBarycenters(entry,subgraphResult)}}});var entries=resolveConflicts(barycenters,cg);expandSubgraphs(entries,subgraphs);var result=sort(entries,biasRight);if(bl){result.vs=_.flatten([bl,result.vs,br],true);if(g.predecessors(bl).length){var blPred=g.node(g.predecessors(bl)[0]),brPred=g.node(g.predecessors(br)[0]);if(!_.has(result,"barycenter")){result.barycenter=0;result.weight=0}result.barycenter=(result.barycenter*result.weight+blPred.order+brPred.order)/(result.weight+2);result.weight+=2}}return result}function expandSubgraphs(entries,subgraphs){_.each(entries,function(entry){entry.vs=_.flatten(entry.vs.map(function(v){if(subgraphs[v]){return subgraphs[v].vs}return v}),true)})}function mergeBarycenters(target,other){if(!_.isUndefined(target.barycenter)){target.barycenter=(target.barycenter*target.weight+other.barycenter*other.weight)/(target.weight+other.weight);target.weight+=other.weight}else{target.barycenter=other.barycenter;target.weight=other.weight}}},{"../lodash":36,"./barycenter":40,"./resolve-conflicts":45,"./sort":47}],47:[function(require,module,exports){var _=require("../lodash"),util=require("../util");module.exports=sort;function sort(entries,biasRight){var parts=util.partition(entries,function(entry){return _.has(entry,"barycenter")});var sortable=parts.lhs,unsortable=_.sortBy(parts.rhs,function(entry){return-entry.i}),vs=[],sum=0,weight=0,vsIndex=0;sortable.sort(compareWithBias(!!biasRight));vsIndex=consumeUnsortable(vs,unsortable,vsIndex);_.each(sortable,function(entry){vsIndex+=entry.vs.length;vs.push(entry.vs);sum+=entry.barycenter*entry.weight;weight+=entry.weight;vsIndex=consumeUnsortable(vs,unsortable,vsIndex)});var result={vs:_.flatten(vs,true)};if(weight){result.barycenter=sum/weight;result.weight=weight}return result}function consumeUnsortable(vs,unsortable,index){var last;while(unsortable.length&&(last=_.last(unsortable)).i<=index){unsortable.pop();vs.push(last.vs);index++}return index}function compareWithBias(bias){return function(entryV,entryW){if(entryV.barycenterentryW.barycenter){return 1}return!bias?entryV.i-entryW.i:entryW.i-entryV.i}}},{"../lodash":36,"../util":55}],48:[function(require,module,exports){var _=require("./lodash");module.exports=parentDummyChains;function parentDummyChains(g){var postorderNums=postorder(g);_.each(g.graph().dummyChains,function(v){var node=g.node(v),edgeObj=node.edgeObj,pathData=findPath(g,postorderNums,edgeObj.v,edgeObj.w),path=pathData.path,lca=pathData.lca,pathIdx=0,pathV=path[pathIdx],ascending=true;while(v!==edgeObj.w){node=g.node(v);if(ascending){while((pathV=path[pathIdx])!==lca&&g.node(pathV).maxRanklow||lim>postorderNums[parent].lim));lca=parent;parent=w;while((parent=g.parent(parent))!==lca){wPath.push(parent)}return{path:vPath.concat(wPath.reverse()),lca:lca}}function postorder(g){var result={},lim=0;function dfs(v){var low=lim;_.each(g.children(v),dfs);result[v]={low:low,lim:lim++}}_.each(g.children(),dfs);return result}},{"./lodash":36}],49:[function(require,module,exports){"use strict";var _=require("../lodash"),Graph=require("../graphlib").Graph,util=require("../util");module.exports={positionX:positionX,findType1Conflicts:findType1Conflicts,findType2Conflicts:findType2Conflicts,addConflict:addConflict,hasConflict:hasConflict,verticalAlignment:verticalAlignment,horizontalCompaction:horizontalCompaction,alignCoordinates:alignCoordinates,findSmallestWidthAlignment:findSmallestWidthAlignment,balance:balance};function findType1Conflicts(g,layering){var conflicts={};function visitLayer(prevLayer,layer){var k0=0,scanPos=0,prevLayerLength=prevLayer.length,lastNode=_.last(layer);_.each(layer,function(v,i){var w=findOtherInnerSegmentNode(g,v),k1=w?g.node(w).order:prevLayerLength;if(w||v===lastNode){_.each(layer.slice(scanPos,i+1),function(scanNode){_.each(g.predecessors(scanNode),function(u){var uLabel=g.node(u),uPos=uLabel.order;if((uPosnextNorthBorder)){addConflict(conflicts,u,v)}})}})}function visitLayer(north,south){var prevNorthPos=-1,nextNorthPos,southPos=0;_.each(south,function(v,southLookahead){if(g.node(v).dummy==="border"){var predecessors=g.predecessors(v);if(predecessors.length){nextNorthPos=g.node(predecessors[0]).order;scan(south,southPos,southLookahead,prevNorthPos,nextNorthPos);southPos=southLookahead;prevNorthPos=nextNorthPos}}scan(south,southPos,south.length,nextNorthPos,north.length)});return south}_.reduce(layering,visitLayer);return conflicts}function findOtherInnerSegmentNode(g,v){if(g.node(v).dummy){return _.find(g.predecessors(v),function(u){return g.node(u).dummy})}}function addConflict(conflicts,v,w){if(v>w){var tmp=v;v=w;w=tmp}var conflictsV=conflicts[v];if(!conflictsV){conflicts[v]=conflictsV={}}conflictsV[w]=true}function hasConflict(conflicts,v,w){if(v>w){var tmp=v;v=w;w=tmp}return _.has(conflicts[v],w)}function verticalAlignment(g,layering,conflicts,neighborFn){var root={},align={},pos={};_.each(layering,function(layer){_.each(layer,function(v,order){root[v]=v;align[v]=v;pos[v]=order})});_.each(layering,function(layer){var prevIdx=-1;_.each(layer,function(v){var ws=neighborFn(v);if(ws.length){ws=_.sortBy(ws,function(w){return pos[w]});var mp=(ws.length-1)/2;for(var i=Math.floor(mp),il=Math.ceil(mp);i<=il;++i){var w=ws[i];if(align[v]===v&&prevIdxwLabel.lim){tailLabel=wLabel;flip=true}var candidates=_.filter(g.edges(),function(edge){return flip===isDescendant(t,t.node(edge.v),tailLabel)&&flip!==isDescendant(t,t.node(edge.w),tailLabel)});return _.min(candidates,function(edge){return slack(g,edge)})}function exchangeEdges(t,g,e,f){var v=e.v,w=e.w;t.removeEdge(v,w);t.setEdge(f.v,f.w,{});initLowLimValues(t);initCutValues(t,g);updateRanks(t,g)}function updateRanks(t,g){var root=_.find(t.nodes(),function(v){return!g.node(v).parent}),vs=preorder(t,root);vs=vs.slice(1);_.each(vs,function(v){var parent=t.node(v).parent,edge=g.edge(v,parent),flipped=false;if(!edge){edge=g.edge(parent,v);flipped=true}g.node(v).rank=g.node(parent).rank+(flipped?edge.minlen:-edge.minlen)})}function isTreeEdge(tree,u,v){return tree.hasEdge(u,v)}function isDescendant(tree,vLabel,rootLabel){return rootLabel.low<=vLabel.lim&&vLabel.lim<=rootLabel.lim}},{"../graphlib":33,"../lodash":36,"../util":55,"./feasible-tree":51,"./util":54}],54:[function(require,module,exports){"use strict";var _=require("../lodash");module.exports={longestPath:longestPath,slack:slack};function longestPath(g){var visited={};function dfs(v){var label=g.node(v);if(_.has(visited,v)){return label.rank}visited[v]=true;var rank=_.min(_.map(g.outEdges(v),function(e){return dfs(e.w)-g.edge(e).minlen}));if(rank===Number.POSITIVE_INFINITY){rank=0}return label.rank=rank}_.each(g.sources(),dfs)}function slack(g,e){return g.node(e.w).rank-g.node(e.v).rank-g.edge(e).minlen}},{"../lodash":36}],55:[function(require,module,exports){"use strict";var _=require("./lodash"),Graph=require("./graphlib").Graph;module.exports={addDummyNode:addDummyNode,simplify:simplify,asNonCompoundGraph:asNonCompoundGraph,successorWeights:successorWeights,predecessorWeights:predecessorWeights,intersectRect:intersectRect,buildLayerMatrix:buildLayerMatrix,normalizeRanks:normalizeRanks,removeEmptyRanks:removeEmptyRanks,addBorderNode:addBorderNode,maxRank:maxRank,partition:partition,time:time,notime:notime};function addDummyNode(g,type,attrs,name){var v;do{v=_.uniqueId(name)}while(g.hasNode(v));attrs.dummy=type;g.setNode(v,attrs);return v}function simplify(g){var simplified=(new Graph).setGraph(g.graph());_.each(g.nodes(),function(v){simplified.setNode(v,g.node(v))});_.each(g.edges(),function(e){var simpleLabel=simplified.edge(e.v,e.w)||{weight:0,minlen:1},label=g.edge(e);simplified.setEdge(e.v,e.w,{weight:simpleLabel.weight+label.weight,minlen:Math.max(simpleLabel.minlen,label.minlen)})});return simplified}function asNonCompoundGraph(g){ -var simplified=new Graph({multigraph:g.isMultigraph()}).setGraph(g.graph());_.each(g.nodes(),function(v){if(!g.children(v).length){simplified.setNode(v,g.node(v))}});_.each(g.edges(),function(e){simplified.setEdge(e,g.edge(e))});return simplified}function successorWeights(g){var weightMap=_.map(g.nodes(),function(v){var sucs={};_.each(g.outEdges(v),function(e){sucs[e.w]=(sucs[e.w]||0)+g.edge(e).weight});return sucs});return _.zipObject(g.nodes(),weightMap)}function predecessorWeights(g){var weightMap=_.map(g.nodes(),function(v){var preds={};_.each(g.inEdges(v),function(e){preds[e.v]=(preds[e.v]||0)+g.edge(e).weight});return preds});return _.zipObject(g.nodes(),weightMap)}function intersectRect(rect,point){var x=rect.x;var y=rect.y;var dx=point.x-x;var dy=point.y-y;var w=rect.width/2;var h=rect.height/2;if(!dx&&!dy){throw new Error("Not possible to find intersection inside of the rectangle")}var sx,sy;if(Math.abs(dy)*w>Math.abs(dx)*h){if(dy<0){h=-h}sx=h*dx/dy;sy=h}else{if(dx<0){w=-w}sx=w;sy=w*dy/dx}return{x:x+sx,y:y+sy}}function buildLayerMatrix(g){var layering=_.map(_.range(maxRank(g)+1),function(){return[]});_.each(g.nodes(),function(v){var node=g.node(v),rank=node.rank;if(!_.isUndefined(rank)){layering[rank][node.order]=v}});return layering}function normalizeRanks(g){var min=_.min(_.map(g.nodes(),function(v){return g.node(v).rank}));_.each(g.nodes(),function(v){var node=g.node(v);if(_.has(node,"rank")){node.rank-=min}})}function removeEmptyRanks(g){var offset=_.min(_.map(g.nodes(),function(v){return g.node(v).rank}));var layers=[];_.each(g.nodes(),function(v){var rank=g.node(v).rank-offset;if(!_.has(layers,rank)){layers[rank]=[]}layers[rank].push(v)});var delta=0,nodeRankFactor=g.graph().nodeRankFactor;_.each(layers,function(vs,i){if(_.isUndefined(vs)&&i%nodeRankFactor!==0){--delta}else if(delta){_.each(vs,function(v){g.node(v).rank+=delta})}})}function addBorderNode(g,prefix,rank,order){var node={width:0,height:0};if(arguments.length>=4){node.rank=rank;node.order=order}return addDummyNode(g,"border",node,prefix)}function maxRank(g){return _.max(_.map(g.nodes(),function(v){var rank=g.node(v).rank;if(!_.isUndefined(rank)){return rank}}))}function partition(collection,fn){var result={lhs:[],rhs:[]};_.each(collection,function(value){if(fn(value)){result.lhs.push(value)}else{result.rhs.push(value)}});return result}function time(name,fn){var start=_.now();try{return fn()}finally{console.log(name+" time: "+(_.now()-start)+"ms")}}function notime(name,fn){return fn()}},{"./graphlib":33,"./lodash":36}],56:[function(require,module,exports){module.exports="0.7.1"},{}],57:[function(require,module,exports){var lib=require("./lib");module.exports={Graph:lib.Graph,json:require("./lib/json"),alg:require("./lib/alg"),version:lib.version}},{"./lib":73,"./lib/alg":64,"./lib/json":74}],58:[function(require,module,exports){var _=require("../lodash");module.exports=components;function components(g){var visited={},cmpts=[],cmpt;function dfs(v){if(_.has(visited,v))return;visited[v]=true;cmpt.push(v);_.each(g.successors(v),dfs);_.each(g.predecessors(v),dfs)}_.each(g.nodes(),function(v){cmpt=[];dfs(v);if(cmpt.length){cmpts.push(cmpt)}});return cmpts}},{"../lodash":75}],59:[function(require,module,exports){var _=require("../lodash");module.exports=dfs;function dfs(g,vs,order){if(!_.isArray(vs)){vs=[vs]}var acc=[],visited={};_.each(vs,function(v){if(!g.hasNode(v)){throw new Error("Graph does not have node: "+v)}doDfs(g,v,order==="post",visited,acc)});return acc}function doDfs(g,v,postorder,visited,acc){if(!_.has(visited,v)){visited[v]=true;if(!postorder){acc.push(v)}_.each(g.neighbors(v),function(w){doDfs(g,w,postorder,visited,acc)});if(postorder){acc.push(v)}}}},{"../lodash":75}],60:[function(require,module,exports){var dijkstra=require("./dijkstra"),_=require("../lodash");module.exports=dijkstraAll;function dijkstraAll(g,weightFunc,edgeFunc){return _.transform(g.nodes(),function(acc,v){acc[v]=dijkstra(g,v,weightFunc,edgeFunc)},{})}},{"../lodash":75,"./dijkstra":61}],61:[function(require,module,exports){var _=require("../lodash"),PriorityQueue=require("../data/priority-queue");module.exports=dijkstra;var DEFAULT_WEIGHT_FUNC=_.constant(1);function dijkstra(g,source,weightFn,edgeFn){return runDijkstra(g,String(source),weightFn||DEFAULT_WEIGHT_FUNC,edgeFn||function(v){return g.outEdges(v)})}function runDijkstra(g,source,weightFn,edgeFn){var results={},pq=new PriorityQueue,v,vEntry;var updateNeighbors=function(edge){var w=edge.v!==v?edge.v:edge.w,wEntry=results[w],weight=weightFn(edge),distance=vEntry.distance+weight;if(weight<0){throw new Error("dijkstra does not allow negative edge weights. "+"Bad edge: "+edge+" Weight: "+weight)}if(distance0){v=pq.removeMin();vEntry=results[v];if(vEntry.distance===Number.POSITIVE_INFINITY){break}edgeFn(v).forEach(updateNeighbors)}return results}},{"../data/priority-queue":71,"../lodash":75}],62:[function(require,module,exports){var _=require("../lodash"),tarjan=require("./tarjan");module.exports=findCycles;function findCycles(g){return _.filter(tarjan(g),function(cmpt){return cmpt.length>1})}},{"../lodash":75,"./tarjan":69}],63:[function(require,module,exports){var _=require("../lodash");module.exports=floydWarshall;var DEFAULT_WEIGHT_FUNC=_.constant(1);function floydWarshall(g,weightFn,edgeFn){return runFloydWarshall(g,weightFn||DEFAULT_WEIGHT_FUNC,edgeFn||function(v){return g.outEdges(v)})}function runFloydWarshall(g,weightFn,edgeFn){var results={},nodes=g.nodes();nodes.forEach(function(v){results[v]={};results[v][v]={distance:0};nodes.forEach(function(w){if(v!==w){results[v][w]={distance:Number.POSITIVE_INFINITY}}});edgeFn(v).forEach(function(edge){var w=edge.v===v?edge.w:edge.v,d=weightFn(edge);results[v][w]={distance:d,predecessor:v}})});nodes.forEach(function(k){var rowK=results[k];nodes.forEach(function(i){var rowI=results[i];nodes.forEach(function(j){var ik=rowI[k];var kj=rowK[j];var ij=rowI[j];var altDistance=ik.distance+kj.distance;if(altDistance0){v=pq.removeMin();if(_.has(parents,v)){result.setEdge(v,parents[v])}else if(init){throw new Error("Input graph is not connected: "+g)}else{init=true}g.nodeEdges(v).forEach(updateNeighbors)}return result}},{"../data/priority-queue":71,"../graph":72,"../lodash":75}],69:[function(require,module,exports){var _=require("../lodash");module.exports=tarjan;function tarjan(g){var index=0,stack=[],visited={},results=[];function dfs(v){var entry=visited[v]={onStack:true,lowlink:index,index:index++};stack.push(v);g.successors(v).forEach(function(w){if(!_.has(visited,w)){dfs(w);entry.lowlink=Math.min(entry.lowlink,visited[w].lowlink)}else if(visited[w].onStack){entry.lowlink=Math.min(entry.lowlink,visited[w].index)}});if(entry.lowlink===entry.index){var cmpt=[],w;do{w=stack.pop();visited[w].onStack=false;cmpt.push(w)}while(v!==w);results.push(cmpt)}}g.nodes().forEach(function(v){if(!_.has(visited,v)){dfs(v)}});return results}},{"../lodash":75}],70:[function(require,module,exports){var _=require("../lodash");module.exports=topsort;topsort.CycleException=CycleException;function topsort(g){var visited={},stack={},results=[];function visit(node){if(_.has(stack,node)){throw new CycleException}if(!_.has(visited,node)){stack[node]=true;visited[node]=true;_.each(g.predecessors(node),visit);delete stack[node];results.push(node)}}_.each(g.sinks(),visit);if(_.size(visited)!==g.nodeCount()){throw new CycleException}return results}function CycleException(){}},{"../lodash":75}],71:[function(require,module,exports){var _=require("../lodash");module.exports=PriorityQueue;function PriorityQueue(){this._arr=[];this._keyIndices={}}PriorityQueue.prototype.size=function(){return this._arr.length};PriorityQueue.prototype.keys=function(){return this._arr.map(function(x){return x.key})};PriorityQueue.prototype.has=function(key){return _.has(this._keyIndices,key)};PriorityQueue.prototype.priority=function(key){var index=this._keyIndices[key];if(index!==undefined){return this._arr[index].priority}};PriorityQueue.prototype.min=function(){if(this.size()===0){throw new Error("Queue underflow")}return this._arr[0].key};PriorityQueue.prototype.add=function(key,priority){var keyIndices=this._keyIndices;key=String(key);if(!_.has(keyIndices,key)){var arr=this._arr;var index=arr.length;keyIndices[key]=index;arr.push({key:key,priority:priority});this._decrease(index);return true}return false};PriorityQueue.prototype.removeMin=function(){this._swap(0,this._arr.length-1);var min=this._arr.pop();delete this._keyIndices[min.key];this._heapify(0);return min.key};PriorityQueue.prototype.decrease=function(key,priority){var index=this._keyIndices[key];if(priority>this._arr[index].priority){throw new Error("New priority is greater than current priority. "+"Key: "+key+" Old: "+this._arr[index].priority+" New: "+priority)}this._arr[index].priority=priority;this._decrease(index)};PriorityQueue.prototype._heapify=function(i){var arr=this._arr;var l=2*i,r=l+1,largest=i;if(l>1;if(arr[parent].priority1){this.setNode(v,value)}else{this.setNode(v)}},this);return this};Graph.prototype.setNode=function(v,value){if(_.has(this._nodes,v)){if(arguments.length>1){this._nodes[v]=value}return this}this._nodes[v]=arguments.length>1?value:this._defaultNodeLabelFn(v);if(this._isCompound){this._parent[v]=GRAPH_NODE;this._children[v]={};this._children[GRAPH_NODE][v]=true}this._in[v]={};this._preds[v]={};this._out[v]={};this._sucs[v]={};++this._nodeCount;return this};Graph.prototype.node=function(v){return this._nodes[v]};Graph.prototype.hasNode=function(v){return _.has(this._nodes,v)};Graph.prototype.removeNode=function(v){var self=this;if(_.has(this._nodes,v)){var removeEdge=function(e){self.removeEdge(self._edgeObjs[e])};delete this._nodes[v];if(this._isCompound){this._removeFromParentsChildList(v);delete this._parent[v];_.each(this.children(v),function(child){this.setParent(child)},this);delete this._children[v]}_.each(_.keys(this._in[v]),removeEdge);delete this._in[v];delete this._preds[v];_.each(_.keys(this._out[v]),removeEdge);delete this._out[v];delete this._sucs[v];--this._nodeCount}return this};Graph.prototype.setParent=function(v,parent){if(!this._isCompound){throw new Error("Cannot set parent in a non-compound graph")}if(_.isUndefined(parent)){parent=GRAPH_NODE}else{for(var ancestor=parent;!_.isUndefined(ancestor);ancestor=this.parent(ancestor)){if(ancestor===v){throw new Error("Setting "+parent+" as parent of "+v+" would create create a cycle")}}this.setNode(parent)}this.setNode(v);this._removeFromParentsChildList(v);this._parent[v]=parent;this._children[parent][v]=true;return this};Graph.prototype._removeFromParentsChildList=function(v){delete this._children[this._parent[v]][v]};Graph.prototype.parent=function(v){if(this._isCompound){var parent=this._parent[v];if(parent!==GRAPH_NODE){return parent}}};Graph.prototype.children=function(v){if(_.isUndefined(v)){v=GRAPH_NODE}if(this._isCompound){var children=this._children[v];if(children){return _.keys(children)}}else if(v===GRAPH_NODE){return this.nodes()}else if(this.hasNode(v)){return[]}};Graph.prototype.predecessors=function(v){var predsV=this._preds[v];if(predsV){return _.keys(predsV)}};Graph.prototype.successors=function(v){var sucsV=this._sucs[v];if(sucsV){return _.keys(sucsV)}};Graph.prototype.neighbors=function(v){var preds=this.predecessors(v);if(preds){return _.union(preds,this.successors(v))}};Graph.prototype.setDefaultEdgeLabel=function(newDefault){if(!_.isFunction(newDefault)){newDefault=_.constant(newDefault)}this._defaultEdgeLabelFn=newDefault;return this};Graph.prototype.edgeCount=function(){return this._edgeCount};Graph.prototype.edges=function(){return _.values(this._edgeObjs)};Graph.prototype.setPath=function(vs,value){var self=this,args=arguments;_.reduce(vs,function(v,w){if(args.length>1){self.setEdge(v,w,value)}else{self.setEdge(v,w)}return w});return this};Graph.prototype.setEdge=function(){var v,w,name,value,valueSpecified=false;if(_.isPlainObject(arguments[0])){v=arguments[0].v;w=arguments[0].w;name=arguments[0].name;if(arguments.length===2){value=arguments[1];valueSpecified=true}}else{v=arguments[0];w=arguments[1];name=arguments[3];if(arguments.length>2){value=arguments[2];valueSpecified=true}}v=""+v;w=""+w;if(!_.isUndefined(name)){name=""+name}var e=edgeArgsToId(this._isDirected,v,w,name);if(_.has(this._edgeLabels,e)){if(valueSpecified){this._edgeLabels[e]=value}return this}if(!_.isUndefined(name)&&!this._isMultigraph){throw new Error("Cannot set a named edge when isMultigraph = false")}this.setNode(v);this.setNode(w);this._edgeLabels[e]=valueSpecified?value:this._defaultEdgeLabelFn(v,w,name);var edgeObj=edgeArgsToObj(this._isDirected,v,w,name);v=edgeObj.v;w=edgeObj.w;Object.freeze(edgeObj);this._edgeObjs[e]=edgeObj;incrementOrInitEntry(this._preds[w],v);incrementOrInitEntry(this._sucs[v],w);this._in[w][e]=edgeObj;this._out[v][e]=edgeObj;this._edgeCount++;return this};Graph.prototype.edge=function(v,w,name){var e=arguments.length===1?edgeObjToId(this._isDirected,arguments[0]):edgeArgsToId(this._isDirected,v,w,name);return this._edgeLabels[e]};Graph.prototype.hasEdge=function(v,w,name){var e=arguments.length===1?edgeObjToId(this._isDirected,arguments[0]):edgeArgsToId(this._isDirected,v,w,name);return _.has(this._edgeLabels,e)};Graph.prototype.removeEdge=function(v,w,name){var e=arguments.length===1?edgeObjToId(this._isDirected,arguments[0]):edgeArgsToId(this._isDirected,v,w,name),edge=this._edgeObjs[e];if(edge){v=edge.v;w=edge.w;delete this._edgeLabels[e];delete this._edgeObjs[e];decrementOrRemoveEntry(this._preds[w],v);decrementOrRemoveEntry(this._sucs[v],w);delete this._in[w][e];delete this._out[v][e];this._edgeCount--}return this};Graph.prototype.inEdges=function(v,u){var inV=this._in[v];if(inV){var edges=_.values(inV);if(!u){return edges}return _.filter(edges,function(edge){return edge.v===u})}};Graph.prototype.outEdges=function(v,w){var outV=this._out[v];if(outV){var edges=_.values(outV);if(!w){return edges}return _.filter(edges,function(edge){return edge.w===w})}};Graph.prototype.nodeEdges=function(v,w){var inEdges=this.inEdges(v,w);if(inEdges){return inEdges.concat(this.outEdges(v,w))}};function incrementOrInitEntry(map,k){if(_.has(map,k)){map[k]++}else{map[k]=1}}function decrementOrRemoveEntry(map,k){if(!--map[k]){delete map[k]}}function edgeArgsToId(isDirected,v,w,name){if(!isDirected&&v>w){var tmp=v;v=w;w=tmp}return v+EDGE_KEY_DELIM+w+EDGE_KEY_DELIM+(_.isUndefined(name)?DEFAULT_EDGE_NAME:name)}function edgeArgsToObj(isDirected,v,w,name){if(!isDirected&&v>w){var tmp=v;v=w;w=tmp}var edgeObj={v:v,w:w};if(name){edgeObj.name=name}return edgeObj}function edgeObjToId(isDirected,edgeObj){return edgeArgsToId(isDirected,edgeObj.v,edgeObj.w,edgeObj.name)}},{"./lodash":75}],73:[function(require,module,exports){module.exports={Graph:require("./graph"),version:require("./version")}},{"./graph":72,"./version":76}],74:[function(require,module,exports){var _=require("./lodash"),Graph=require("./graph");module.exports={write:write,read:read};function write(g){var json={options:{directed:g.isDirected(),multigraph:g.isMultigraph(),compound:g.isCompound()},nodes:writeNodes(g),edges:writeEdges(g)};if(!_.isUndefined(g.graph())){json.value=_.clone(g.graph())}return json}function writeNodes(g){return _.map(g.nodes(),function(v){var nodeValue=g.node(v),parent=g.parent(v),node={v:v};if(!_.isUndefined(nodeValue)){node.value=nodeValue}if(!_.isUndefined(parent)){node.parent=parent}return node})}function writeEdges(g){return _.map(g.edges(),function(e){var edgeValue=g.edge(e),edge={v:e.v,w:e.w};if(!_.isUndefined(e.name)){edge.name=e.name}if(!_.isUndefined(edgeValue)){edge.value=edgeValue}return edge})}function read(json){var g=new Graph(json.options).setGraph(json.value);_.each(json.nodes,function(entry){g.setNode(entry.v,entry.value);if(entry.parent){g.setParent(entry.v,entry.parent)}});_.each(json.edges,function(entry){g.setEdge({v:entry.v,w:entry.w,name:entry.name},entry.value)});return g}},{"./graph":72,"./lodash":75}],75:[function(require,module,exports){arguments[4][20][0].apply(exports,arguments)},{dup:20,lodash:77}],76:[function(require,module,exports){module.exports="1.0.1"},{}],77:[function(require,module,exports){(function(global){(function(){var undefined;var arrayPool=[],objectPool=[];var idCounter=0;var keyPrefix=+new Date+"";var largeArraySize=75;var maxPoolSize=40;var whitespace=" \f \ufeff"+"\n\r\u2028\u2029"+" ᠎              ";var reEmptyStringLeading=/\b__p \+= '';/g,reEmptyStringMiddle=/\b(__p \+=) '' \+/g,reEmptyStringTrailing=/(__e\(.*?\)|\b__t\)) \+\n'';/g;var reEsTemplate=/\$\{([^\\}]*(?:\\.[^\\}]*)*)\}/g;var reFlags=/\w*$/;var reFuncName=/^\s*function[ \n\r\t]+\w/;var reInterpolate=/<%=([\s\S]+?)%>/g;var reLeadingSpacesAndZeros=RegExp("^["+whitespace+"]*0+(?=.$)");var reNoMatch=/($^)/;var reThis=/\bthis\b/;var reUnescapedString=/['\n\r\t\u2028\u2029\\]/g;var contextProps=["Array","Boolean","Date","Function","Math","Number","Object","RegExp","String","_","attachEvent","clearTimeout","isFinite","isNaN","parseInt","setTimeout"];var templateCounter=0;var argsClass="[object Arguments]",arrayClass="[object Array]",boolClass="[object Boolean]",dateClass="[object Date]",funcClass="[object Function]",numberClass="[object Number]",objectClass="[object Object]",regexpClass="[object RegExp]",stringClass="[object String]";var cloneableClasses={};cloneableClasses[funcClass]=false;cloneableClasses[argsClass]=cloneableClasses[arrayClass]=cloneableClasses[boolClass]=cloneableClasses[dateClass]=cloneableClasses[numberClass]=cloneableClasses[objectClass]=cloneableClasses[regexpClass]=cloneableClasses[stringClass]=true;var debounceOptions={leading:false,maxWait:0,trailing:false};var descriptor={configurable:false,enumerable:false,value:null,writable:false};var objectTypes={"boolean":false,"function":true,object:true,number:false,string:false,undefined:false};var stringEscapes={"\\":"\\","'":"'","\n":"n","\r":"r"," ":"t","\u2028":"u2028","\u2029":"u2029"};var root=objectTypes[typeof window]&&window||this;var freeExports=objectTypes[typeof exports]&&exports&&!exports.nodeType&&exports;var freeModule=objectTypes[typeof module]&&module&&!module.nodeType&&module;var moduleExports=freeModule&&freeModule.exports===freeExports&&freeExports;var freeGlobal=objectTypes[typeof global]&&global;if(freeGlobal&&(freeGlobal.global===freeGlobal||freeGlobal.window===freeGlobal)){root=freeGlobal}function baseIndexOf(array,value,fromIndex){var index=(fromIndex||0)-1,length=array?array.length:0;while(++index-1?0:-1:cache?0:-1}function cachePush(value){var cache=this.cache,type=typeof value;if(type=="boolean"||value==null){cache[value]=true}else{if(type!="number"&&type!="string"){type="object"}var key=type=="number"?value:keyPrefix+value,typeCache=cache[type]||(cache[type]={});if(type=="object"){(typeCache[key]||(typeCache[key]=[])).push(value)}else{typeCache[key]=true}}}function charAtCallback(value){return value.charCodeAt(0)}function compareAscending(a,b){var ac=a.criteria,bc=b.criteria,index=-1,length=ac.length;while(++indexother||typeof value=="undefined"){return 1}if(value/g,evaluate:/<%([\s\S]+?)%>/g,interpolate:reInterpolate,variable:"",imports:{_:lodash}};function baseBind(bindData){var func=bindData[0],partialArgs=bindData[2],thisArg=bindData[4];function bound(){if(partialArgs){var args=slice(partialArgs);push.apply(args,arguments)}if(this instanceof bound){var thisBinding=baseCreate(func.prototype),result=func.apply(thisBinding,args||arguments);return isObject(result)?result:thisBinding}return func.apply(thisArg,args||arguments)}setBindData(bound,bindData);return bound}function baseClone(value,isDeep,callback,stackA,stackB){if(callback){var result=callback(value);if(typeof result!="undefined"){return result}}var isObj=isObject(value);if(isObj){var className=toString.call(value);if(!cloneableClasses[className]){return value}var ctor=ctorByClass[className];switch(className){case boolClass:case dateClass:return new ctor(+value);case numberClass:case stringClass:return new ctor(value);case regexpClass:result=ctor(value.source,reFlags.exec(value));result.lastIndex=value.lastIndex;return result}}else{return value}var isArr=isArray(value);if(isDeep){var initedStack=!stackA;stackA||(stackA=getArray());stackB||(stackB=getArray());var length=stackA.length;while(length--){if(stackA[length]==value){return stackB[length]}}result=isArr?ctor(value.length):{}}else{result=isArr?slice(value):assign({},value)}if(isArr){if(hasOwnProperty.call(value,"index")){result.index=value.index}if(hasOwnProperty.call(value,"input")){result.input=value.input}}if(!isDeep){return result}stackA.push(value);stackB.push(result);(isArr?forEach:forOwn)(value,function(objValue,key){result[key]=baseClone(objValue,isDeep,callback,stackA,stackB)});if(initedStack){releaseArray(stackA);releaseArray(stackB)}return result}function baseCreate(prototype,properties){return isObject(prototype)?nativeCreate(prototype):{}}if(!nativeCreate){baseCreate=function(){function Object(){}return function(prototype){if(isObject(prototype)){Object.prototype=prototype;var result=new Object;Object.prototype=null}return result||context.Object()}}()}function baseCreateCallback(func,thisArg,argCount){if(typeof func!="function"){return identity}if(typeof thisArg=="undefined"||!("prototype"in func)){return func}var bindData=func.__bindData__;if(typeof bindData=="undefined"){if(support.funcNames){bindData=!func.name}bindData=bindData||!support.funcDecomp;if(!bindData){var source=fnToString.call(func);if(!support.funcNames){bindData=!reFuncName.test(source)}if(!bindData){bindData=reThis.test(source);setBindData(func,bindData)}}}if(bindData===false||bindData!==true&&bindData[1]&1){return func}switch(argCount){case 1:return function(value){return func.call(thisArg,value)};case 2:return function(a,b){return func.call(thisArg,a,b)};case 3:return function(value,index,collection){return func.call(thisArg,value,index,collection)};case 4:return function(accumulator,value,index,collection){return func.call(thisArg,accumulator,value,index,collection)}}return bind(func,thisArg)}function baseCreateWrapper(bindData){var func=bindData[0],bitmask=bindData[1],partialArgs=bindData[2],partialRightArgs=bindData[3],thisArg=bindData[4],arity=bindData[5];var isBind=bitmask&1,isBindKey=bitmask&2,isCurry=bitmask&4,isCurryBound=bitmask&8,key=func;function bound(){var thisBinding=isBind?thisArg:this;if(partialArgs){var args=slice(partialArgs);push.apply(args,arguments)}if(partialRightArgs||isCurry){args||(args=slice(arguments));if(partialRightArgs){push.apply(args,partialRightArgs)}if(isCurry&&args.length=largeArraySize&&indexOf===baseIndexOf,result=[];if(isLarge){var cache=createCache(values);if(cache){indexOf=cacheIndexOf;values=cache}else{isLarge=false}}while(++index-1}})}}stackA.pop();stackB.pop();if(initedStack){releaseArray(stackA);releaseArray(stackB)}return result}function baseMerge(object,source,callback,stackA,stackB){(isArray(source)?forEach:forOwn)(source,function(source,key){var found,isArr,result=source,value=object[key];if(source&&((isArr=isArray(source))||isPlainObject(source))){var stackLength=stackA.length;while(stackLength--){if(found=stackA[stackLength]==source){value=stackB[stackLength];break}}if(!found){var isShallow;if(callback){result=callback(value,source);if(isShallow=typeof result!="undefined"){value=result}}if(!isShallow){value=isArr?isArray(value)?value:[]:isPlainObject(value)?value:{}}stackA.push(source);stackB.push(value);if(!isShallow){baseMerge(value,source,callback,stackA,stackB)}}}else{if(callback){result=callback(value,source);if(typeof result=="undefined"){result=source}}if(typeof result!="undefined"){value=result}}object[key]=value})}function baseRandom(min,max){return min+floor(nativeRandom()*(max-min+1))}function baseUniq(array,isSorted,callback){var index=-1,indexOf=getIndexOf(),length=array?array.length:0,result=[];var isLarge=!isSorted&&length>=largeArraySize&&indexOf===baseIndexOf,seen=callback||isLarge?getArray():result;if(isLarge){var cache=createCache(seen);indexOf=cacheIndexOf;seen=cache}while(++index":">",'"':""","'":"'"};var htmlUnescapes=invert(htmlEscapes);var reEscapedHtml=RegExp("("+keys(htmlUnescapes).join("|")+")","g"),reUnescapedHtml=RegExp("["+keys(htmlEscapes).join("")+"]","g");var assign=function(object,source,guard){var index,iterable=object,result=iterable;if(!iterable)return result;var args=arguments,argsIndex=0,argsLength=typeof guard=="number"?2:args.length;if(argsLength>3&&typeof args[argsLength-2]=="function"){var callback=baseCreateCallback(args[--argsLength-1],args[argsLength--],2)}else if(argsLength>2&&typeof args[argsLength-1]=="function"){callback=args[--argsLength]}while(++argsIndex3&&typeof args[length-2]=="function"){var callback=baseCreateCallback(args[--length-1],args[length--],2)}else if(length>2&&typeof args[length-1]=="function"){callback=args[--length]}var sources=slice(arguments,1,length),index=-1,stackA=getArray(),stackB=getArray();while(++index-1}else if(typeof length=="number"){result=(isString(collection)?collection.indexOf(target,fromIndex):indexOf(collection,target,fromIndex))>-1}else{forOwn(collection,function(value){if(++index>=fromIndex){return!(result=value===target)}})}return result}var countBy=createAggregator(function(result,value,key){hasOwnProperty.call(result,key)?result[key]++:result[key]=1});function every(collection,callback,thisArg){var result=true;callback=lodash.createCallback(callback,thisArg,3);var index=-1,length=collection?collection.length:0;if(typeof length=="number"){while(++indexresult){result=value}}}else{callback=callback==null&&isString(collection)?charAtCallback:lodash.createCallback(callback,thisArg,3);forEach(collection,function(value,index,collection){var current=callback(value,index,collection);if(current>computed){computed=current;result=value}})}return result}function min(collection,callback,thisArg){var computed=Infinity,result=computed;if(typeof callback!="function"&&thisArg&&thisArg[callback]===collection){callback=null}if(callback==null&&isArray(collection)){var index=-1,length=collection.length;while(++index=largeArraySize&&createCache(argsIndex?args[argsIndex]:seen))}}var array=args[0],index=-1,length=array?array.length:0,result=[];outer:while(++index>>1;callback(array[mid])1?arguments:arguments[0],index=-1,length=array?max(pluck(array,"length")):0,result=Array(length<0?0:length);while(++index2?createWrapper(func,17,slice(arguments,2),null,thisArg):createWrapper(func,1,null,null,thisArg)}function bindAll(object){var funcs=arguments.length>1?baseFlatten(arguments,true,false,1):functions(object),index=-1,length=funcs.length;while(++index2?createWrapper(key,19,slice(arguments,2),null,object):createWrapper(key,3,null,null,object)}function compose(){var funcs=arguments,length=funcs.length;while(length--){if(!isFunction(funcs[length])){throw new TypeError}}return function(){var args=arguments,length=funcs.length;while(length--){args=[funcs[length].apply(this,args)]}return args[0]}}function curry(func,arity){arity=typeof arity=="number"?arity:+arity||func.length;return createWrapper(func,4,null,null,null,arity)}function debounce(func,wait,options){var args,maxTimeoutId,result,stamp,thisArg,timeoutId,trailingCall,lastCalled=0,maxWait=false,trailing=true;if(!isFunction(func)){throw new TypeError}wait=nativeMax(0,wait)||0;if(options===true){var leading=true;trailing=false}else if(isObject(options)){leading=options.leading;maxWait="maxWait"in options&&(nativeMax(wait,options.maxWait)||0);trailing="trailing"in options?options.trailing:trailing}var delayed=function(){var remaining=wait-(now()-stamp);if(remaining<=0){if(maxTimeoutId){clearTimeout(maxTimeoutId)}var isCalled=trailingCall;maxTimeoutId=timeoutId=trailingCall=undefined;if(isCalled){lastCalled=now();result=func.apply(thisArg,args);if(!timeoutId&&!maxTimeoutId){args=thisArg=null}}}else{timeoutId=setTimeout(delayed,remaining)}};var maxDelayed=function(){if(timeoutId){clearTimeout(timeoutId)}maxTimeoutId=timeoutId=trailingCall=undefined;if(trailing||maxWait!==wait){lastCalled=now();result=func.apply(thisArg,args);if(!timeoutId&&!maxTimeoutId){args=thisArg=null}}};return function(){args=arguments; +_.each(inputGraph.nodes(),function(v){var node=canonicalize(inputGraph.node(v));g.setNode(v,_.defaults(selectNumberAttrs(node,nodeNumAttrs),nodeDefaults));g.setParent(v,inputGraph.parent(v))});_.each(inputGraph.edges(),function(e){var edge=canonicalize(inputGraph.edge(e));g.setEdge(e,_.merge({},edgeDefaults,selectNumberAttrs(edge,edgeNumAttrs),_.pick(edge,edgeAttrs)))});return g}function makeSpaceForEdgeLabels(g){var graph=g.graph();graph.ranksep/=2;_.each(g.edges(),function(e){var edge=g.edge(e);edge.minlen*=2;if(edge.labelpos.toLowerCase()!=="c"){if(graph.rankdir==="TB"||graph.rankdir==="BT"){edge.width+=edge.labeloffset}else{edge.height+=edge.labeloffset}}})}function injectEdgeLabelProxies(g){_.each(g.edges(),function(e){var edge=g.edge(e);if(edge.width&&edge.height){var v=g.node(e.v),w=g.node(e.w),label={rank:(w.rank-v.rank)/2+v.rank,e:e};util.addDummyNode(g,"edge-proxy",label,"_ep")}})}function assignRankMinMax(g){var maxRank=0;_.each(g.nodes(),function(v){var node=g.node(v);if(node.borderTop){node.minRank=g.node(node.borderTop).rank;node.maxRank=g.node(node.borderBottom).rank;maxRank=_.max(maxRank,node.maxRank)}});g.graph().maxRank=maxRank}function removeEdgeLabelProxies(g){_.each(g.nodes(),function(v){var node=g.node(v);if(node.dummy==="edge-proxy"){g.edge(node.e).labelRank=node.rank;g.removeNode(v)}})}function translateGraph(g){var minX=Number.POSITIVE_INFINITY,maxX=0,minY=Number.POSITIVE_INFINITY,maxY=0,graphLabel=g.graph(),marginX=graphLabel.marginx||0,marginY=graphLabel.marginy||0;function getExtremes(attrs){var x=attrs.x,y=attrs.y,w=attrs.width,h=attrs.height;minX=Math.min(minX,x-w/2);maxX=Math.max(maxX,x+w/2);minY=Math.min(minY,y-h/2);maxY=Math.max(maxY,y+h/2)}_.each(g.nodes(),function(v){getExtremes(g.node(v))});_.each(g.edges(),function(e){var edge=g.edge(e);if(_.has(edge,"x")){getExtremes(edge)}});minX-=marginX;minY-=marginY;_.each(g.nodes(),function(v){var node=g.node(v);node.x-=minX;node.y-=minY});_.each(g.edges(),function(e){var edge=g.edge(e);_.each(edge.points,function(p){p.x-=minX;p.y-=minY});if(_.has(edge,"x")){edge.x-=minX}if(_.has(edge,"y")){edge.y-=minY}});graphLabel.width=maxX-minX+marginX;graphLabel.height=maxY-minY+marginY}function assignNodeIntersects(g){_.each(g.edges(),function(e){var edge=g.edge(e),nodeV=g.node(e.v),nodeW=g.node(e.w),p1,p2;if(!edge.points){edge.points=[];p1=nodeW;p2=nodeV}else{p1=edge.points[0];p2=edge.points[edge.points.length-1]}edge.points.unshift(util.intersectRect(nodeV,p1));edge.points.push(util.intersectRect(nodeW,p2))})}function fixupEdgeLabelCoords(g){_.each(g.edges(),function(e){var edge=g.edge(e);if(_.has(edge,"x")){if(edge.labelpos==="l"||edge.labelpos==="r"){edge.width-=edge.labeloffset}switch(edge.labelpos){case"l":edge.x-=edge.width/2+edge.labeloffset;break;case"r":edge.x+=edge.width/2+edge.labeloffset;break}}})}function reversePointsForReversedEdges(g){_.each(g.edges(),function(e){var edge=g.edge(e);if(edge.reversed){edge.points.reverse()}})}function removeBorderNodes(g){_.each(g.nodes(),function(v){if(g.children(v).length){var node=g.node(v),t=g.node(node.borderTop),b=g.node(node.borderBottom),l=g.node(_.last(node.borderLeft)),r=g.node(_.last(node.borderRight));node.width=Math.abs(r.x-l.x);node.height=Math.abs(b.y-t.y);node.x=l.x+node.width/2;node.y=t.y+node.height/2}});_.each(g.nodes(),function(v){if(g.node(v).dummy==="border"){g.removeNode(v)}})}function removeSelfEdges(g){_.each(g.edges(),function(e){if(e.v===e.w){var node=g.node(e.v);if(!node.selfEdges){node.selfEdges=[]}node.selfEdges.push({e:e,label:g.edge(e)});g.removeEdge(e)}})}function insertSelfEdges(g){var layers=util.buildLayerMatrix(g);_.each(layers,function(layer){var orderShift=0;_.each(layer,function(v,i){var node=g.node(v);node.order=i+orderShift;_.each(node.selfEdges,function(selfEdge){util.addDummyNode(g,"selfedge",{width:selfEdge.label.width,height:selfEdge.label.height,rank:node.rank,order:i+ ++orderShift,e:selfEdge.e,label:selfEdge.label},"_se")});delete node.selfEdges})})}function positionSelfEdges(g){_.each(g.nodes(),function(v){var node=g.node(v);if(node.dummy==="selfedge"){var selfNode=g.node(node.e.v),x=selfNode.x+selfNode.width/2,y=selfNode.y,dx=node.x-x,dy=selfNode.height/2;g.setEdge(node.e,node.label);g.removeNode(v);node.label.points=[{x:x+2*dx/3,y:y-dy},{x:x+5*dx/6,y:y-dy},{x:x+dx,y:y},{x:x+5*dx/6,y:y+dy},{x:x+2*dx/3,y:y+dy}];node.label.x=node.x;node.label.y=node.y}})}function selectNumberAttrs(obj,attrs){return _.mapValues(_.pick(obj,attrs),Number)}function canonicalize(attrs){var newAttrs={};_.each(attrs,function(v,k){newAttrs[k.toLowerCase()]=v});return newAttrs}},{"./acyclic":28,"./add-border-segments":29,"./coordinate-system":30,"./graphlib":33,"./lodash":36,"./nesting-graph":37,"./normalize":38,"./order":43,"./parent-dummy-chains":48,"./position":50,"./rank":52,"./util":55}],36:[function(require,module,exports){module.exports=require(20)},{"/Users/andrew/Documents/dev/dagre-d3/lib/lodash.js":20,lodash:77}],37:[function(require,module,exports){var _=require("./lodash"),util=require("./util");module.exports={run:run,cleanup:cleanup};function run(g){var root=util.addDummyNode(g,"root",{},"_root"),depths=treeDepths(g),height=_.max(depths)-1,nodeSep=2*height+1;g.graph().nestingRoot=root;_.each(g.edges(),function(e){g.edge(e).minlen*=nodeSep});var weight=sumWeights(g)+1;_.each(g.children(),function(child){dfs(g,root,nodeSep,weight,height,depths,child)});g.graph().nodeRankFactor=nodeSep}function dfs(g,root,nodeSep,weight,height,depths,v){var children=g.children(v);if(!children.length){if(v!==root){g.setEdge(root,v,{weight:0,minlen:nodeSep})}return}var top=util.addBorderNode(g,"_bt"),bottom=util.addBorderNode(g,"_bb"),label=g.node(v);g.setParent(top,v);label.borderTop=top;g.setParent(bottom,v);label.borderBottom=bottom;_.each(children,function(child){dfs(g,root,nodeSep,weight,height,depths,child);var childNode=g.node(child),childTop=childNode.borderTop?childNode.borderTop:child,childBottom=childNode.borderBottom?childNode.borderBottom:child,thisWeight=childNode.borderTop?weight:2*weight,minlen=childTop!==childBottom?1:height-depths[v]+1;g.setEdge(top,childTop,{weight:thisWeight,minlen:minlen,nestingEdge:true});g.setEdge(childBottom,bottom,{weight:thisWeight,minlen:minlen,nestingEdge:true})});if(!g.parent(v)){g.setEdge(root,top,{weight:0,minlen:height+depths[v]})}}function treeDepths(g){var depths={};function dfs(v,depth){var children=g.children(v);if(children&&children.length){_.each(children,function(child){dfs(child,depth+1)})}depths[v]=depth}_.each(g.children(),function(v){dfs(v,1)});return depths}function sumWeights(g){return _.reduce(g.edges(),function(acc,e){return acc+g.edge(e).weight},0)}function cleanup(g){var graphLabel=g.graph();g.removeNode(graphLabel.nestingRoot);delete graphLabel.nestingRoot;_.each(g.edges(),function(e){var edge=g.edge(e);if(edge.nestingEdge){g.removeEdge(e)}})}},{"./lodash":36,"./util":55}],38:[function(require,module,exports){"use strict";var _=require("./lodash"),util=require("./util");module.exports={run:run,undo:undo};function run(g){g.graph().dummyChains=[];_.each(g.edges(),function(edge){normalizeEdge(g,edge)})}function normalizeEdge(g,e){var v=e.v,vRank=g.node(v).rank,w=e.w,wRank=g.node(w).rank,name=e.name,edgeLabel=g.edge(e),labelRank=edgeLabel.labelRank;if(wRank===vRank+1)return;g.removeEdge(e);var dummy,attrs,i;for(i=0,++vRank;vRank0){if(index%2){weightSum+=tree[index+1]}index=index-1>>1;tree[index]+=entry.weight}cc+=entry.weight*weightSum}));return cc}},{"../lodash":36}],43:[function(require,module,exports){"use strict";var _=require("../lodash"),initOrder=require("./init-order"),crossCount=require("./cross-count"),sortSubgraph=require("./sort-subgraph"),buildLayerGraph=require("./build-layer-graph"),addSubgraphConstraints=require("./add-subgraph-constraints"),Graph=require("../graphlib").Graph,util=require("../util");module.exports=order;function order(g){var maxRank=util.maxRank(g),downLayerGraphs=buildLayerGraphs(g,_.range(1,maxRank+1),"inEdges"),upLayerGraphs=buildLayerGraphs(g,_.range(maxRank-1,-1,-1),"outEdges");var layering=initOrder(g);assignOrder(g,layering);var bestCC=Number.POSITIVE_INFINITY,best;for(var i=0,lastBest=0;lastBest<4;++i,++lastBest){sweepLayerGraphs(i%2?downLayerGraphs:upLayerGraphs,i%4>=2);layering=util.buildLayerMatrix(g);var cc=crossCount(g,layering);if(cc=vEntry.barycenter){mergeEntries(vEntry,uEntry)}}}function handleOut(vEntry){return function(wEntry){wEntry["in"].push(vEntry);if(--wEntry.indegree===0){sourceSet.push(wEntry)}}}while(sourceSet.length){var entry=sourceSet.pop();entries.push(entry);_.each(entry["in"].reverse(),handleIn(entry));_.each(entry.out,handleOut(entry))}return _.chain(entries).filter(function(entry){return!entry.merged}).map(function(entry){return _.pick(entry,["vs","i","barycenter","weight"])}).value()}function mergeEntries(target,source){var sum=0,weight=0;if(target.weight){sum+=target.barycenter*target.weight;weight+=target.weight}if(source.weight){sum+=source.barycenter*source.weight;weight+=source.weight}target.vs=source.vs.concat(target.vs);target.barycenter=sum/weight;target.weight=weight;target.i=Math.min(source.i,target.i);source.merged=true}},{"../lodash":36}],46:[function(require,module,exports){var _=require("../lodash"),barycenter=require("./barycenter"),resolveConflicts=require("./resolve-conflicts"),sort=require("./sort");module.exports=sortSubgraph;function sortSubgraph(g,v,cg,biasRight){var movable=g.children(v),node=g.node(v),bl=node?node.borderLeft:undefined,br=node?node.borderRight:undefined,subgraphs={};if(bl){movable=_.filter(movable,function(w){return w!==bl&&w!==br})}var barycenters=barycenter(g,movable);_.each(barycenters,function(entry){if(g.children(entry.v).length){var subgraphResult=sortSubgraph(g,entry.v,cg,biasRight);subgraphs[entry.v]=subgraphResult;if(_.has(subgraphResult,"barycenter")){mergeBarycenters(entry,subgraphResult)}}});var entries=resolveConflicts(barycenters,cg);expandSubgraphs(entries,subgraphs);var result=sort(entries,biasRight);if(bl){result.vs=_.flatten([bl,result.vs,br],true);if(g.predecessors(bl).length){var blPred=g.node(g.predecessors(bl)[0]),brPred=g.node(g.predecessors(br)[0]);if(!_.has(result,"barycenter")){result.barycenter=0;result.weight=0}result.barycenter=(result.barycenter*result.weight+blPred.order+brPred.order)/(result.weight+2);result.weight+=2}}return result}function expandSubgraphs(entries,subgraphs){_.each(entries,function(entry){entry.vs=_.flatten(entry.vs.map(function(v){if(subgraphs[v]){return subgraphs[v].vs}return v}),true)})}function mergeBarycenters(target,other){if(!_.isUndefined(target.barycenter)){target.barycenter=(target.barycenter*target.weight+other.barycenter*other.weight)/(target.weight+other.weight);target.weight+=other.weight}else{target.barycenter=other.barycenter;target.weight=other.weight}}},{"../lodash":36,"./barycenter":40,"./resolve-conflicts":45,"./sort":47}],47:[function(require,module,exports){var _=require("../lodash"),util=require("../util");module.exports=sort;function sort(entries,biasRight){var parts=util.partition(entries,function(entry){return _.has(entry,"barycenter")});var sortable=parts.lhs,unsortable=_.sortBy(parts.rhs,function(entry){return-entry.i}),vs=[],sum=0,weight=0,vsIndex=0;sortable.sort(compareWithBias(!!biasRight));vsIndex=consumeUnsortable(vs,unsortable,vsIndex);_.each(sortable,function(entry){vsIndex+=entry.vs.length;vs.push(entry.vs);sum+=entry.barycenter*entry.weight;weight+=entry.weight;vsIndex=consumeUnsortable(vs,unsortable,vsIndex)});var result={vs:_.flatten(vs,true)};if(weight){result.barycenter=sum/weight;result.weight=weight}return result}function consumeUnsortable(vs,unsortable,index){var last;while(unsortable.length&&(last=_.last(unsortable)).i<=index){unsortable.pop();vs.push(last.vs);index++}return index}function compareWithBias(bias){return function(entryV,entryW){if(entryV.barycenterentryW.barycenter){return 1}return!bias?entryV.i-entryW.i:entryW.i-entryV.i}}},{"../lodash":36,"../util":55}],48:[function(require,module,exports){var _=require("./lodash");module.exports=parentDummyChains;function parentDummyChains(g){var postorderNums=postorder(g);_.each(g.graph().dummyChains,function(v){var node=g.node(v),edgeObj=node.edgeObj,pathData=findPath(g,postorderNums,edgeObj.v,edgeObj.w),path=pathData.path,lca=pathData.lca,pathIdx=0,pathV=path[pathIdx],ascending=true;while(v!==edgeObj.w){node=g.node(v);if(ascending){while((pathV=path[pathIdx])!==lca&&g.node(pathV).maxRanklow||lim>postorderNums[parent].lim));lca=parent;parent=w;while((parent=g.parent(parent))!==lca){wPath.push(parent)}return{path:vPath.concat(wPath.reverse()),lca:lca}}function postorder(g){var result={},lim=0;function dfs(v){var low=lim;_.each(g.children(v),dfs);result[v]={low:low,lim:lim++}}_.each(g.children(),dfs);return result}},{"./lodash":36}],49:[function(require,module,exports){"use strict";var _=require("../lodash"),Graph=require("../graphlib").Graph,util=require("../util");module.exports={positionX:positionX,findType1Conflicts:findType1Conflicts,findType2Conflicts:findType2Conflicts,addConflict:addConflict,hasConflict:hasConflict,verticalAlignment:verticalAlignment,horizontalCompaction:horizontalCompaction,alignCoordinates:alignCoordinates,findSmallestWidthAlignment:findSmallestWidthAlignment,balance:balance};function findType1Conflicts(g,layering){var conflicts={};function visitLayer(prevLayer,layer){var k0=0,scanPos=0,prevLayerLength=prevLayer.length,lastNode=_.last(layer);_.each(layer,function(v,i){var w=findOtherInnerSegmentNode(g,v),k1=w?g.node(w).order:prevLayerLength;if(w||v===lastNode){_.each(layer.slice(scanPos,i+1),function(scanNode){_.each(g.predecessors(scanNode),function(u){var uLabel=g.node(u),uPos=uLabel.order;if((uPosnextNorthBorder)){addConflict(conflicts,u,v)}})}})}function visitLayer(north,south){var prevNorthPos=-1,nextNorthPos,southPos=0;_.each(south,function(v,southLookahead){if(g.node(v).dummy==="border"){var predecessors=g.predecessors(v);if(predecessors.length){nextNorthPos=g.node(predecessors[0]).order;scan(south,southPos,southLookahead,prevNorthPos,nextNorthPos);southPos=southLookahead;prevNorthPos=nextNorthPos}}scan(south,southPos,south.length,nextNorthPos,north.length)});return south}_.reduce(layering,visitLayer);return conflicts}function findOtherInnerSegmentNode(g,v){if(g.node(v).dummy){return _.find(g.predecessors(v),function(u){return g.node(u).dummy})}}function addConflict(conflicts,v,w){if(v>w){var tmp=v;v=w;w=tmp}var conflictsV=conflicts[v];if(!conflictsV){conflicts[v]=conflictsV={}}conflictsV[w]=true}function hasConflict(conflicts,v,w){if(v>w){var tmp=v;v=w;w=tmp}return _.has(conflicts[v],w)}function verticalAlignment(g,layering,conflicts,neighborFn){var root={},align={},pos={};_.each(layering,function(layer){_.each(layer,function(v,order){root[v]=v;align[v]=v;pos[v]=order})});_.each(layering,function(layer){var prevIdx=-1;_.each(layer,function(v){var ws=neighborFn(v);if(ws.length){ws=_.sortBy(ws,function(w){return pos[w]});var mp=(ws.length-1)/2;for(var i=Math.floor(mp),il=Math.ceil(mp);i<=il;++i){var w=ws[i];if(align[v]===v&&prevIdxwLabel.lim){tailLabel=wLabel;flip=true}var candidates=_.filter(g.edges(),function(edge){return flip===isDescendant(t,t.node(edge.v),tailLabel)&&flip!==isDescendant(t,t.node(edge.w),tailLabel)});return _.min(candidates,function(edge){return slack(g,edge)})}function exchangeEdges(t,g,e,f){var v=e.v,w=e.w;t.removeEdge(v,w);t.setEdge(f.v,f.w,{});initLowLimValues(t);initCutValues(t,g);updateRanks(t,g)}function updateRanks(t,g){var root=_.find(t.nodes(),function(v){return!g.node(v).parent}),vs=preorder(t,root);vs=vs.slice(1);_.each(vs,function(v){var parent=t.node(v).parent,edge=g.edge(v,parent),flipped=false;if(!edge){edge=g.edge(parent,v);flipped=true}g.node(v).rank=g.node(parent).rank+(flipped?edge.minlen:-edge.minlen)})}function isTreeEdge(tree,u,v){return tree.hasEdge(u,v)}function isDescendant(tree,vLabel,rootLabel){return rootLabel.low<=vLabel.lim&&vLabel.lim<=rootLabel.lim}},{"../graphlib":33,"../lodash":36,"../util":55,"./feasible-tree":51,"./util":54}],54:[function(require,module,exports){"use strict";var _=require("../lodash");module.exports={longestPath:longestPath,slack:slack};function longestPath(g){var visited={};function dfs(v){var label=g.node(v);if(_.has(visited,v)){return label.rank}visited[v]=true;var rank=_.min(_.map(g.outEdges(v),function(e){return dfs(e.w)-g.edge(e).minlen}));if(rank===Number.POSITIVE_INFINITY){rank=0}return label.rank=rank}_.each(g.sources(),dfs)}function slack(g,e){return g.node(e.w).rank-g.node(e.v).rank-g.edge(e).minlen}},{"../lodash":36}],55:[function(require,module,exports){"use strict";var _=require("./lodash"),Graph=require("./graphlib").Graph;module.exports={addDummyNode:addDummyNode,simplify:simplify,asNonCompoundGraph:asNonCompoundGraph,successorWeights:successorWeights,predecessorWeights:predecessorWeights,intersectRect:intersectRect,buildLayerMatrix:buildLayerMatrix,normalizeRanks:normalizeRanks,removeEmptyRanks:removeEmptyRanks,addBorderNode:addBorderNode,maxRank:maxRank,partition:partition,time:time,notime:notime};function addDummyNode(g,type,attrs,name){var v; -stamp=now();thisArg=this;trailingCall=trailing&&(timeoutId||!leading);if(maxWait===false){var leadingCall=leading&&!timeoutId}else{if(!maxTimeoutId&&!leading){lastCalled=stamp}var remaining=maxWait-(stamp-lastCalled),isCalled=remaining<=0;if(isCalled){if(maxTimeoutId){maxTimeoutId=clearTimeout(maxTimeoutId)}lastCalled=stamp;result=func.apply(thisArg,args)}else if(!maxTimeoutId){maxTimeoutId=setTimeout(maxDelayed,remaining)}}if(isCalled&&timeoutId){timeoutId=clearTimeout(timeoutId)}else if(!timeoutId&&wait!==maxWait){timeoutId=setTimeout(delayed,wait)}if(leadingCall){isCalled=true;result=func.apply(thisArg,args)}if(isCalled&&!timeoutId&&!maxTimeoutId){args=thisArg=null}return result}}function defer(func){if(!isFunction(func)){throw new TypeError}var args=slice(arguments,1);return setTimeout(function(){func.apply(undefined,args)},1)}function delay(func,wait){if(!isFunction(func)){throw new TypeError}var args=slice(arguments,2);return setTimeout(function(){func.apply(undefined,args)},wait)}function memoize(func,resolver){if(!isFunction(func)){throw new TypeError}var memoized=function(){var cache=memoized.cache,key=resolver?resolver.apply(this,arguments):keyPrefix+arguments[0];return hasOwnProperty.call(cache,key)?cache[key]:cache[key]=func.apply(this,arguments)};memoized.cache={};return memoized}function once(func){var ran,result;if(!isFunction(func)){throw new TypeError}return function(){if(ran){return result}ran=true;result=func.apply(this,arguments);func=null;return result}}function partial(func){return createWrapper(func,16,slice(arguments,1))}function partialRight(func){return createWrapper(func,32,null,slice(arguments,1))}function throttle(func,wait,options){var leading=true,trailing=true;if(!isFunction(func)){throw new TypeError}if(options===false){leading=false}else if(isObject(options)){leading="leading"in options?options.leading:leading;trailing="trailing"in options?options.trailing:trailing}debounceOptions.leading=leading;debounceOptions.maxWait=wait;debounceOptions.trailing=trailing;return debounce(func,wait,debounceOptions)}function wrap(value,wrapper){return createWrapper(wrapper,16,[value])}function constant(value){return function(){return value}}function createCallback(func,thisArg,argCount){var type=typeof func;if(func==null||type=="function"){return baseCreateCallback(func,thisArg,argCount)}if(type!="object"){return property(func)}var props=keys(func),key=props[0],a=func[key];if(props.length==1&&a===a&&!isObject(a)){return function(object){var b=object[key];return a===b&&(a!==0||1/a==1/b)}}return function(object){var length=props.length,result=false;while(length--){if(!(result=baseIsEqual(object[props[length]],func[props[length]],null,true))){break}}return result}}function escape(string){return string==null?"":String(string).replace(reUnescapedHtml,escapeHtmlChar)}function identity(value){return value}function mixin(object,source,options){var chain=true,methodNames=source&&functions(source);if(!source||!options&&!methodNames.length){if(options==null){options=source}ctor=lodashWrapper;source=object;object=lodash;methodNames=functions(source)}if(options===false){chain=false}else if(isObject(options)&&"chain"in options){chain=options.chain}var ctor=object,isFunc=isFunction(ctor);forEach(methodNames,function(methodName){var func=object[methodName]=source[methodName];if(isFunc){ctor.prototype[methodName]=function(){var chainAll=this.__chain__,value=this.__wrapped__,args=[value];push.apply(args,arguments);var result=func.apply(object,args);if(chain||chainAll){if(value===result&&isObject(result)){return this}result=new ctor(result);result.__chain__=chainAll}return result}}})}function noConflict(){context._=oldDash;return this}function noop(){}var now=isNative(now=Date.now)&&now||function(){return(new Date).getTime()};var parseInt=nativeParseInt(whitespace+"08")==8?nativeParseInt:function(value,radix){return nativeParseInt(isString(value)?value.replace(reLeadingSpacesAndZeros,""):value,radix||0)};function property(key){return function(object){return object[key]}}function random(min,max,floating){var noMin=min==null,noMax=max==null;if(floating==null){if(typeof min=="boolean"&&noMax){floating=min;min=1}else if(!noMax&&typeof max=="boolean"){floating=max;noMax=true}}if(noMin&&noMax){max=1}min=+min||0;if(noMax){max=min;min=0}else{max=+max||0}if(floating||min%1||max%1){var rand=nativeRandom();return nativeMin(min+rand*(max-min+parseFloat("1e-"+((rand+"").length-1))),max)}return baseRandom(min,max)}function result(object,key){if(object){var value=object[key];return isFunction(value)?object[key]():value}}function template(text,data,options){var settings=lodash.templateSettings;text=String(text||"");options=defaults({},options,settings);var imports=defaults({},options.imports,settings.imports),importsKeys=keys(imports),importsValues=values(imports);var isEvaluating,index=0,interpolate=options.interpolate||reNoMatch,source="__p += '";var reDelimiters=RegExp((options.escape||reNoMatch).source+"|"+interpolate.source+"|"+(interpolate===reInterpolate?reEsTemplate:reNoMatch).source+"|"+(options.evaluate||reNoMatch).source+"|$","g");text.replace(reDelimiters,function(match,escapeValue,interpolateValue,esTemplateValue,evaluateValue,offset){interpolateValue||(interpolateValue=esTemplateValue);source+=text.slice(index,offset).replace(reUnescapedString,escapeStringChar);if(escapeValue){source+="' +\n__e("+escapeValue+") +\n'"}if(evaluateValue){isEvaluating=true;source+="';\n"+evaluateValue+";\n__p += '"}if(interpolateValue){source+="' +\n((__t = ("+interpolateValue+")) == null ? '' : __t) +\n'"}index=offset+match.length;return match});source+="';\n";var variable=options.variable,hasVariable=variable;if(!hasVariable){variable="obj";source="with ("+variable+") {\n"+source+"\n}\n"}source=(isEvaluating?source.replace(reEmptyStringLeading,""):source).replace(reEmptyStringMiddle,"$1").replace(reEmptyStringTrailing,"$1;");source="function("+variable+") {\n"+(hasVariable?"":variable+" || ("+variable+" = {});\n")+"var __t, __p = '', __e = _.escape"+(isEvaluating?", __j = Array.prototype.join;\n"+"function print() { __p += __j.call(arguments, '') }\n":";\n")+source+"return __p\n}";var sourceURL="\n/*\n//# sourceURL="+(options.sourceURL||"/lodash/template/source["+templateCounter++ +"]")+"\n*/";try{var result=Function(importsKeys,"return "+source+sourceURL).apply(undefined,importsValues)}catch(e){e.source=source;throw e}if(data){return result(data)}result.source=source;return result}function times(n,callback,thisArg){n=(n=+n)>-1?n:0;var index=-1,result=Array(n);callback=baseCreateCallback(callback,thisArg,1);while(++indexMath.abs(dx)*h){if(dy<0){h=-h}sx=h*dx/dy;sy=h}else{if(dx<0){w=-w}sx=w;sy=w*dy/dx}return{x:x+sx,y:y+sy}}function buildLayerMatrix(g){var layering=_.map(_.range(maxRank(g)+1),function(){return[]});_.each(g.nodes(),function(v){var node=g.node(v),rank=node.rank;if(!_.isUndefined(rank)){layering[rank][node.order]=v}});return layering}function normalizeRanks(g){var min=_.min(_.map(g.nodes(),function(v){return g.node(v).rank}));_.each(g.nodes(),function(v){var node=g.node(v);if(_.has(node,"rank")){node.rank-=min}})}function removeEmptyRanks(g){var offset=_.min(_.map(g.nodes(),function(v){return g.node(v).rank}));var layers=[];_.each(g.nodes(),function(v){var rank=g.node(v).rank-offset;if(!_.has(layers,rank)){layers[rank]=[]}layers[rank].push(v)});var delta=0,nodeRankFactor=g.graph().nodeRankFactor;_.each(layers,function(vs,i){if(_.isUndefined(vs)&&i%nodeRankFactor!==0){--delta}else if(delta){_.each(vs,function(v){g.node(v).rank+=delta})}})}function addBorderNode(g,prefix,rank,order){var node={width:0,height:0};if(arguments.length>=4){node.rank=rank;node.order=order}return addDummyNode(g,"border",node,prefix)}function maxRank(g){return _.max(_.map(g.nodes(),function(v){var rank=g.node(v).rank;if(!_.isUndefined(rank)){return rank}}))}function partition(collection,fn){var result={lhs:[],rhs:[]};_.each(collection,function(value){if(fn(value)){result.lhs.push(value)}else{result.rhs.push(value)}});return result}function time(name,fn){var start=_.now();try{return fn()}finally{console.log(name+" time: "+(_.now()-start)+"ms")}}function notime(name,fn){return fn()}},{"./graphlib":33,"./lodash":36}],56:[function(require,module,exports){module.exports="0.7.1"},{}],57:[function(require,module,exports){var lib=require("./lib");module.exports={Graph:lib.Graph,json:require("./lib/json"),alg:require("./lib/alg"),version:lib.version}},{"./lib":73,"./lib/alg":64,"./lib/json":74}],58:[function(require,module,exports){var _=require("../lodash");module.exports=components;function components(g){var visited={},cmpts=[],cmpt;function dfs(v){if(_.has(visited,v))return;visited[v]=true;cmpt.push(v);_.each(g.successors(v),dfs);_.each(g.predecessors(v),dfs)}_.each(g.nodes(),function(v){cmpt=[];dfs(v);if(cmpt.length){cmpts.push(cmpt)}});return cmpts}},{"../lodash":75}],59:[function(require,module,exports){var _=require("../lodash");module.exports=dfs;function dfs(g,vs,order){if(!_.isArray(vs)){vs=[vs]}var acc=[],visited={};_.each(vs,function(v){if(!g.hasNode(v)){throw new Error("Graph does not have node: "+v)}doDfs(g,v,order==="post",visited,acc)});return acc}function doDfs(g,v,postorder,visited,acc){if(!_.has(visited,v)){visited[v]=true;if(!postorder){acc.push(v)}_.each(g.neighbors(v),function(w){doDfs(g,w,postorder,visited,acc)});if(postorder){acc.push(v)}}}},{"../lodash":75}],60:[function(require,module,exports){var dijkstra=require("./dijkstra"),_=require("../lodash");module.exports=dijkstraAll;function dijkstraAll(g,weightFunc,edgeFunc){return _.transform(g.nodes(),function(acc,v){acc[v]=dijkstra(g,v,weightFunc,edgeFunc)},{})}},{"../lodash":75,"./dijkstra":61}],61:[function(require,module,exports){var _=require("../lodash"),PriorityQueue=require("../data/priority-queue");module.exports=dijkstra;var DEFAULT_WEIGHT_FUNC=_.constant(1);function dijkstra(g,source,weightFn,edgeFn){return runDijkstra(g,String(source),weightFn||DEFAULT_WEIGHT_FUNC,edgeFn||function(v){return g.outEdges(v)})}function runDijkstra(g,source,weightFn,edgeFn){var results={},pq=new PriorityQueue,v,vEntry;var updateNeighbors=function(edge){var w=edge.v!==v?edge.v:edge.w,wEntry=results[w],weight=weightFn(edge),distance=vEntry.distance+weight;if(weight<0){throw new Error("dijkstra does not allow negative edge weights. "+"Bad edge: "+edge+" Weight: "+weight)}if(distance0){v=pq.removeMin();vEntry=results[v];if(vEntry.distance===Number.POSITIVE_INFINITY){break}edgeFn(v).forEach(updateNeighbors)}return results}},{"../data/priority-queue":71,"../lodash":75}],62:[function(require,module,exports){var _=require("../lodash"),tarjan=require("./tarjan");module.exports=findCycles;function findCycles(g){return _.filter(tarjan(g),function(cmpt){return cmpt.length>1})}},{"../lodash":75,"./tarjan":69}],63:[function(require,module,exports){var _=require("../lodash");module.exports=floydWarshall;var DEFAULT_WEIGHT_FUNC=_.constant(1);function floydWarshall(g,weightFn,edgeFn){return runFloydWarshall(g,weightFn||DEFAULT_WEIGHT_FUNC,edgeFn||function(v){return g.outEdges(v)})}function runFloydWarshall(g,weightFn,edgeFn){var results={},nodes=g.nodes();nodes.forEach(function(v){results[v]={};results[v][v]={distance:0};nodes.forEach(function(w){if(v!==w){results[v][w]={distance:Number.POSITIVE_INFINITY}}});edgeFn(v).forEach(function(edge){var w=edge.v===v?edge.w:edge.v,d=weightFn(edge);results[v][w]={distance:d,predecessor:v}})});nodes.forEach(function(k){var rowK=results[k];nodes.forEach(function(i){var rowI=results[i];nodes.forEach(function(j){var ik=rowI[k];var kj=rowK[j];var ij=rowI[j];var altDistance=ik.distance+kj.distance;if(altDistance0){v=pq.removeMin();if(_.has(parents,v)){result.setEdge(v,parents[v])}else if(init){throw new Error("Input graph is not connected: "+g)}else{init=true}g.nodeEdges(v).forEach(updateNeighbors)}return result}},{"../data/priority-queue":71,"../graph":72,"../lodash":75}],69:[function(require,module,exports){var _=require("../lodash");module.exports=tarjan;function tarjan(g){var index=0,stack=[],visited={},results=[];function dfs(v){var entry=visited[v]={onStack:true,lowlink:index,index:index++};stack.push(v);g.successors(v).forEach(function(w){if(!_.has(visited,w)){dfs(w);entry.lowlink=Math.min(entry.lowlink,visited[w].lowlink)}else if(visited[w].onStack){entry.lowlink=Math.min(entry.lowlink,visited[w].index)}});if(entry.lowlink===entry.index){var cmpt=[],w;do{w=stack.pop();visited[w].onStack=false;cmpt.push(w)}while(v!==w);results.push(cmpt)}}g.nodes().forEach(function(v){if(!_.has(visited,v)){dfs(v)}});return results}},{"../lodash":75}],70:[function(require,module,exports){var _=require("../lodash");module.exports=topsort;topsort.CycleException=CycleException;function topsort(g){var visited={},stack={},results=[];function visit(node){if(_.has(stack,node)){throw new CycleException}if(!_.has(visited,node)){stack[node]=true;visited[node]=true;_.each(g.predecessors(node),visit);delete stack[node];results.push(node)}}_.each(g.sinks(),visit);if(_.size(visited)!==g.nodeCount()){throw new CycleException}return results}function CycleException(){}},{"../lodash":75}],71:[function(require,module,exports){var _=require("../lodash");module.exports=PriorityQueue;function PriorityQueue(){this._arr=[];this._keyIndices={}}PriorityQueue.prototype.size=function(){return this._arr.length};PriorityQueue.prototype.keys=function(){return this._arr.map(function(x){return x.key})};PriorityQueue.prototype.has=function(key){return _.has(this._keyIndices,key)};PriorityQueue.prototype.priority=function(key){var index=this._keyIndices[key];if(index!==undefined){return this._arr[index].priority}};PriorityQueue.prototype.min=function(){if(this.size()===0){throw new Error("Queue underflow")}return this._arr[0].key};PriorityQueue.prototype.add=function(key,priority){var keyIndices=this._keyIndices;key=String(key);if(!_.has(keyIndices,key)){var arr=this._arr;var index=arr.length;keyIndices[key]=index;arr.push({key:key,priority:priority});this._decrease(index);return true}return false};PriorityQueue.prototype.removeMin=function(){this._swap(0,this._arr.length-1);var min=this._arr.pop();delete this._keyIndices[min.key];this._heapify(0);return min.key};PriorityQueue.prototype.decrease=function(key,priority){var index=this._keyIndices[key];if(priority>this._arr[index].priority){throw new Error("New priority is greater than current priority. "+"Key: "+key+" Old: "+this._arr[index].priority+" New: "+priority)}this._arr[index].priority=priority;this._decrease(index)};PriorityQueue.prototype._heapify=function(i){var arr=this._arr;var l=2*i,r=l+1,largest=i;if(l>1;if(arr[parent].priority1){this.setNode(v,value)}else{this.setNode(v)}},this);return this};Graph.prototype.setNode=function(v,value){if(_.has(this._nodes,v)){if(arguments.length>1){this._nodes[v]=value}return this}this._nodes[v]=arguments.length>1?value:this._defaultNodeLabelFn(v);if(this._isCompound){this._parent[v]=GRAPH_NODE;this._children[v]={};this._children[GRAPH_NODE][v]=true}this._in[v]={};this._preds[v]={};this._out[v]={};this._sucs[v]={};++this._nodeCount;return this};Graph.prototype.node=function(v){return this._nodes[v]};Graph.prototype.hasNode=function(v){return _.has(this._nodes,v)};Graph.prototype.removeNode=function(v){var self=this;if(_.has(this._nodes,v)){var removeEdge=function(e){self.removeEdge(self._edgeObjs[e])};delete this._nodes[v];if(this._isCompound){this._removeFromParentsChildList(v);delete this._parent[v];_.each(this.children(v),function(child){this.setParent(child)},this);delete this._children[v]}_.each(_.keys(this._in[v]),removeEdge);delete this._in[v];delete this._preds[v];_.each(_.keys(this._out[v]),removeEdge);delete this._out[v];delete this._sucs[v];--this._nodeCount}return this};Graph.prototype.setParent=function(v,parent){if(!this._isCompound){throw new Error("Cannot set parent in a non-compound graph")}if(_.isUndefined(parent)){parent=GRAPH_NODE}else{for(var ancestor=parent;!_.isUndefined(ancestor);ancestor=this.parent(ancestor)){if(ancestor===v){throw new Error("Setting "+parent+" as parent of "+v+" would create create a cycle")}}this.setNode(parent)}this.setNode(v);this._removeFromParentsChildList(v);this._parent[v]=parent;this._children[parent][v]=true;return this};Graph.prototype._removeFromParentsChildList=function(v){delete this._children[this._parent[v]][v]};Graph.prototype.parent=function(v){if(this._isCompound){var parent=this._parent[v];if(parent!==GRAPH_NODE){return parent}}};Graph.prototype.children=function(v){if(_.isUndefined(v)){v=GRAPH_NODE}if(this._isCompound){var children=this._children[v];if(children){return _.keys(children)}}else if(v===GRAPH_NODE){return this.nodes()}else if(this.hasNode(v)){return[]}};Graph.prototype.predecessors=function(v){var predsV=this._preds[v];if(predsV){return _.keys(predsV)}};Graph.prototype.successors=function(v){var sucsV=this._sucs[v];if(sucsV){return _.keys(sucsV)}};Graph.prototype.neighbors=function(v){var preds=this.predecessors(v);if(preds){return _.union(preds,this.successors(v))}};Graph.prototype.setDefaultEdgeLabel=function(newDefault){if(!_.isFunction(newDefault)){newDefault=_.constant(newDefault)}this._defaultEdgeLabelFn=newDefault;return this};Graph.prototype.edgeCount=function(){return this._edgeCount};Graph.prototype.edges=function(){return _.values(this._edgeObjs)};Graph.prototype.setPath=function(vs,value){var self=this,args=arguments;_.reduce(vs,function(v,w){if(args.length>1){self.setEdge(v,w,value)}else{self.setEdge(v,w)}return w});return this};Graph.prototype.setEdge=function(){var v,w,name,value,valueSpecified=false;if(_.isPlainObject(arguments[0])){v=arguments[0].v;w=arguments[0].w;name=arguments[0].name;if(arguments.length===2){value=arguments[1];valueSpecified=true}}else{v=arguments[0];w=arguments[1];name=arguments[3];if(arguments.length>2){value=arguments[2];valueSpecified=true}}v=""+v;w=""+w;if(!_.isUndefined(name)){name=""+name}var e=edgeArgsToId(this._isDirected,v,w,name);if(_.has(this._edgeLabels,e)){if(valueSpecified){this._edgeLabels[e]=value}return this}if(!_.isUndefined(name)&&!this._isMultigraph){throw new Error("Cannot set a named edge when isMultigraph = false")}this.setNode(v);this.setNode(w);this._edgeLabels[e]=valueSpecified?value:this._defaultEdgeLabelFn(v,w,name);var edgeObj=edgeArgsToObj(this._isDirected,v,w,name);v=edgeObj.v;w=edgeObj.w;Object.freeze(edgeObj);this._edgeObjs[e]=edgeObj;incrementOrInitEntry(this._preds[w],v);incrementOrInitEntry(this._sucs[v],w);this._in[w][e]=edgeObj;this._out[v][e]=edgeObj;this._edgeCount++;return this};Graph.prototype.edge=function(v,w,name){var e=arguments.length===1?edgeObjToId(this._isDirected,arguments[0]):edgeArgsToId(this._isDirected,v,w,name);return this._edgeLabels[e]};Graph.prototype.hasEdge=function(v,w,name){var e=arguments.length===1?edgeObjToId(this._isDirected,arguments[0]):edgeArgsToId(this._isDirected,v,w,name);return _.has(this._edgeLabels,e)};Graph.prototype.removeEdge=function(v,w,name){var e=arguments.length===1?edgeObjToId(this._isDirected,arguments[0]):edgeArgsToId(this._isDirected,v,w,name),edge=this._edgeObjs[e];if(edge){v=edge.v;w=edge.w;delete this._edgeLabels[e];delete this._edgeObjs[e];decrementOrRemoveEntry(this._preds[w],v);decrementOrRemoveEntry(this._sucs[v],w);delete this._in[w][e];delete this._out[v][e];this._edgeCount--}return this};Graph.prototype.inEdges=function(v,u){var inV=this._in[v];if(inV){var edges=_.values(inV);if(!u){return edges}return _.filter(edges,function(edge){return edge.v===u})}};Graph.prototype.outEdges=function(v,w){var outV=this._out[v];if(outV){var edges=_.values(outV);if(!w){return edges}return _.filter(edges,function(edge){return edge.w===w})}};Graph.prototype.nodeEdges=function(v,w){var inEdges=this.inEdges(v,w);if(inEdges){return inEdges.concat(this.outEdges(v,w))}};function incrementOrInitEntry(map,k){if(_.has(map,k)){map[k]++}else{map[k]=1}}function decrementOrRemoveEntry(map,k){if(!--map[k]){delete map[k]}}function edgeArgsToId(isDirected,v,w,name){if(!isDirected&&v>w){var tmp=v;v=w;w=tmp}return v+EDGE_KEY_DELIM+w+EDGE_KEY_DELIM+(_.isUndefined(name)?DEFAULT_EDGE_NAME:name)}function edgeArgsToObj(isDirected,v,w,name){if(!isDirected&&v>w){var tmp=v;v=w;w=tmp}var edgeObj={v:v,w:w};if(name){edgeObj.name=name}return edgeObj}function edgeObjToId(isDirected,edgeObj){return edgeArgsToId(isDirected,edgeObj.v,edgeObj.w,edgeObj.name)}},{"./lodash":75}],73:[function(require,module,exports){module.exports={Graph:require("./graph"),version:require("./version")}},{"./graph":72,"./version":76}],74:[function(require,module,exports){var _=require("./lodash"),Graph=require("./graph");module.exports={write:write,read:read};function write(g){var json={options:{directed:g.isDirected(),multigraph:g.isMultigraph(),compound:g.isCompound()},nodes:writeNodes(g),edges:writeEdges(g)};if(!_.isUndefined(g.graph())){json.value=_.clone(g.graph())}return json}function writeNodes(g){return _.map(g.nodes(),function(v){var nodeValue=g.node(v),parent=g.parent(v),node={v:v};if(!_.isUndefined(nodeValue)){node.value=nodeValue}if(!_.isUndefined(parent)){node.parent=parent}return node})}function writeEdges(g){return _.map(g.edges(),function(e){var edgeValue=g.edge(e),edge={v:e.v,w:e.w};if(!_.isUndefined(e.name)){edge.name=e.name}if(!_.isUndefined(edgeValue)){edge.value=edgeValue}return edge})}function read(json){var g=new Graph(json.options).setGraph(json.value);_.each(json.nodes,function(entry){g.setNode(entry.v,entry.value);if(entry.parent){g.setParent(entry.v,entry.parent)}});_.each(json.edges,function(entry){g.setEdge({v:entry.v,w:entry.w,name:entry.name},entry.value)});return g}},{"./graph":72,"./lodash":75}],75:[function(require,module,exports){module.exports=require(20)},{"/Users/andrew/Documents/dev/dagre-d3/lib/lodash.js":20,lodash:77}],76:[function(require,module,exports){module.exports="1.0.1"},{}],77:[function(require,module,exports){(function(global){(function(){var undefined;var arrayPool=[],objectPool=[];var idCounter=0;var keyPrefix=+new Date+"";var largeArraySize=75;var maxPoolSize=40;var whitespace=" \f \ufeff"+"\n\r\u2028\u2029"+" ᠎              ";var reEmptyStringLeading=/\b__p \+= '';/g,reEmptyStringMiddle=/\b(__p \+=) '' \+/g,reEmptyStringTrailing=/(__e\(.*?\)|\b__t\)) \+\n'';/g;var reEsTemplate=/\$\{([^\\}]*(?:\\.[^\\}]*)*)\}/g;var reFlags=/\w*$/;var reFuncName=/^\s*function[ \n\r\t]+\w/;var reInterpolate=/<%=([\s\S]+?)%>/g;var reLeadingSpacesAndZeros=RegExp("^["+whitespace+"]*0+(?=.$)");var reNoMatch=/($^)/;var reThis=/\bthis\b/;var reUnescapedString=/['\n\r\t\u2028\u2029\\]/g;var contextProps=["Array","Boolean","Date","Function","Math","Number","Object","RegExp","String","_","attachEvent","clearTimeout","isFinite","isNaN","parseInt","setTimeout"];var templateCounter=0;var argsClass="[object Arguments]",arrayClass="[object Array]",boolClass="[object Boolean]",dateClass="[object Date]",funcClass="[object Function]",numberClass="[object Number]",objectClass="[object Object]",regexpClass="[object RegExp]",stringClass="[object String]";var cloneableClasses={};cloneableClasses[funcClass]=false;cloneableClasses[argsClass]=cloneableClasses[arrayClass]=cloneableClasses[boolClass]=cloneableClasses[dateClass]=cloneableClasses[numberClass]=cloneableClasses[objectClass]=cloneableClasses[regexpClass]=cloneableClasses[stringClass]=true;var debounceOptions={leading:false,maxWait:0,trailing:false};var descriptor={configurable:false,enumerable:false,value:null,writable:false};var objectTypes={"boolean":false,"function":true,object:true,number:false,string:false,undefined:false};var stringEscapes={"\\":"\\","'":"'","\n":"n","\r":"r"," ":"t","\u2028":"u2028","\u2029":"u2029"};var root=objectTypes[typeof window]&&window||this;var freeExports=objectTypes[typeof exports]&&exports&&!exports.nodeType&&exports;var freeModule=objectTypes[typeof module]&&module&&!module.nodeType&&module;var moduleExports=freeModule&&freeModule.exports===freeExports&&freeExports;var freeGlobal=objectTypes[typeof global]&&global;if(freeGlobal&&(freeGlobal.global===freeGlobal||freeGlobal.window===freeGlobal)){root=freeGlobal}function baseIndexOf(array,value,fromIndex){var index=(fromIndex||0)-1,length=array?array.length:0;while(++index-1?0:-1:cache?0:-1}function cachePush(value){var cache=this.cache,type=typeof value;if(type=="boolean"||value==null){cache[value]=true}else{if(type!="number"&&type!="string"){type="object"}var key=type=="number"?value:keyPrefix+value,typeCache=cache[type]||(cache[type]={});if(type=="object"){(typeCache[key]||(typeCache[key]=[])).push(value)}else{typeCache[key]=true}}}function charAtCallback(value){return value.charCodeAt(0)}function compareAscending(a,b){var ac=a.criteria,bc=b.criteria,index=-1,length=ac.length;while(++indexother||typeof value=="undefined"){return 1}if(value/g,evaluate:/<%([\s\S]+?)%>/g,interpolate:reInterpolate,variable:"",imports:{_:lodash}};function baseBind(bindData){var func=bindData[0],partialArgs=bindData[2],thisArg=bindData[4];function bound(){if(partialArgs){var args=slice(partialArgs);push.apply(args,arguments)}if(this instanceof bound){var thisBinding=baseCreate(func.prototype),result=func.apply(thisBinding,args||arguments);return isObject(result)?result:thisBinding}return func.apply(thisArg,args||arguments)}setBindData(bound,bindData);return bound}function baseClone(value,isDeep,callback,stackA,stackB){if(callback){var result=callback(value);if(typeof result!="undefined"){return result}}var isObj=isObject(value);if(isObj){var className=toString.call(value);if(!cloneableClasses[className]){return value}var ctor=ctorByClass[className];switch(className){case boolClass:case dateClass:return new ctor(+value);case numberClass:case stringClass:return new ctor(value);case regexpClass:result=ctor(value.source,reFlags.exec(value));result.lastIndex=value.lastIndex;return result}}else{return value}var isArr=isArray(value);if(isDeep){var initedStack=!stackA;stackA||(stackA=getArray());stackB||(stackB=getArray());var length=stackA.length;while(length--){if(stackA[length]==value){return stackB[length]}}result=isArr?ctor(value.length):{}}else{result=isArr?slice(value):assign({},value)}if(isArr){if(hasOwnProperty.call(value,"index")){result.index=value.index}if(hasOwnProperty.call(value,"input")){result.input=value.input}}if(!isDeep){return result}stackA.push(value);stackB.push(result);(isArr?forEach:forOwn)(value,function(objValue,key){result[key]=baseClone(objValue,isDeep,callback,stackA,stackB)});if(initedStack){releaseArray(stackA);releaseArray(stackB)}return result}function baseCreate(prototype,properties){return isObject(prototype)?nativeCreate(prototype):{}}if(!nativeCreate){baseCreate=function(){function Object(){}return function(prototype){if(isObject(prototype)){Object.prototype=prototype;var result=new Object;Object.prototype=null}return result||context.Object()}}()}function baseCreateCallback(func,thisArg,argCount){if(typeof func!="function"){return identity}if(typeof thisArg=="undefined"||!("prototype"in func)){return func}var bindData=func.__bindData__;if(typeof bindData=="undefined"){if(support.funcNames){bindData=!func.name}bindData=bindData||!support.funcDecomp;if(!bindData){var source=fnToString.call(func);if(!support.funcNames){bindData=!reFuncName.test(source)}if(!bindData){bindData=reThis.test(source);setBindData(func,bindData)}}}if(bindData===false||bindData!==true&&bindData[1]&1){return func}switch(argCount){case 1:return function(value){return func.call(thisArg,value)};case 2:return function(a,b){return func.call(thisArg,a,b)};case 3:return function(value,index,collection){return func.call(thisArg,value,index,collection)};case 4:return function(accumulator,value,index,collection){return func.call(thisArg,accumulator,value,index,collection)}}return bind(func,thisArg)}function baseCreateWrapper(bindData){var func=bindData[0],bitmask=bindData[1],partialArgs=bindData[2],partialRightArgs=bindData[3],thisArg=bindData[4],arity=bindData[5];var isBind=bitmask&1,isBindKey=bitmask&2,isCurry=bitmask&4,isCurryBound=bitmask&8,key=func;function bound(){var thisBinding=isBind?thisArg:this;if(partialArgs){var args=slice(partialArgs);push.apply(args,arguments)}if(partialRightArgs||isCurry){args||(args=slice(arguments));if(partialRightArgs){push.apply(args,partialRightArgs)}if(isCurry&&args.length=largeArraySize&&indexOf===baseIndexOf,result=[]; + +if(isLarge){var cache=createCache(values);if(cache){indexOf=cacheIndexOf;values=cache}else{isLarge=false}}while(++index-1}})}}stackA.pop();stackB.pop();if(initedStack){releaseArray(stackA);releaseArray(stackB)}return result}function baseMerge(object,source,callback,stackA,stackB){(isArray(source)?forEach:forOwn)(source,function(source,key){var found,isArr,result=source,value=object[key];if(source&&((isArr=isArray(source))||isPlainObject(source))){var stackLength=stackA.length;while(stackLength--){if(found=stackA[stackLength]==source){value=stackB[stackLength];break}}if(!found){var isShallow;if(callback){result=callback(value,source);if(isShallow=typeof result!="undefined"){value=result}}if(!isShallow){value=isArr?isArray(value)?value:[]:isPlainObject(value)?value:{}}stackA.push(source);stackB.push(value);if(!isShallow){baseMerge(value,source,callback,stackA,stackB)}}}else{if(callback){result=callback(value,source);if(typeof result=="undefined"){result=source}}if(typeof result!="undefined"){value=result}}object[key]=value})}function baseRandom(min,max){return min+floor(nativeRandom()*(max-min+1))}function baseUniq(array,isSorted,callback){var index=-1,indexOf=getIndexOf(),length=array?array.length:0,result=[];var isLarge=!isSorted&&length>=largeArraySize&&indexOf===baseIndexOf,seen=callback||isLarge?getArray():result;if(isLarge){var cache=createCache(seen);indexOf=cacheIndexOf;seen=cache}while(++index":">",'"':""","'":"'"};var htmlUnescapes=invert(htmlEscapes);var reEscapedHtml=RegExp("("+keys(htmlUnescapes).join("|")+")","g"),reUnescapedHtml=RegExp("["+keys(htmlEscapes).join("")+"]","g");var assign=function(object,source,guard){var index,iterable=object,result=iterable;if(!iterable)return result;var args=arguments,argsIndex=0,argsLength=typeof guard=="number"?2:args.length;if(argsLength>3&&typeof args[argsLength-2]=="function"){var callback=baseCreateCallback(args[--argsLength-1],args[argsLength--],2)}else if(argsLength>2&&typeof args[argsLength-1]=="function"){callback=args[--argsLength]}while(++argsIndex3&&typeof args[length-2]=="function"){var callback=baseCreateCallback(args[--length-1],args[length--],2)}else if(length>2&&typeof args[length-1]=="function"){callback=args[--length]}var sources=slice(arguments,1,length),index=-1,stackA=getArray(),stackB=getArray();while(++index-1}else if(typeof length=="number"){result=(isString(collection)?collection.indexOf(target,fromIndex):indexOf(collection,target,fromIndex))>-1}else{forOwn(collection,function(value){if(++index>=fromIndex){return!(result=value===target)}})}return result}var countBy=createAggregator(function(result,value,key){hasOwnProperty.call(result,key)?result[key]++:result[key]=1});function every(collection,callback,thisArg){var result=true;callback=lodash.createCallback(callback,thisArg,3);var index=-1,length=collection?collection.length:0;if(typeof length=="number"){while(++indexresult){result=value}}}else{callback=callback==null&&isString(collection)?charAtCallback:lodash.createCallback(callback,thisArg,3);forEach(collection,function(value,index,collection){var current=callback(value,index,collection);if(current>computed){computed=current;result=value}})}return result}function min(collection,callback,thisArg){var computed=Infinity,result=computed;if(typeof callback!="function"&&thisArg&&thisArg[callback]===collection){callback=null}if(callback==null&&isArray(collection)){var index=-1,length=collection.length;while(++index=largeArraySize&&createCache(argsIndex?args[argsIndex]:seen))}}var array=args[0],index=-1,length=array?array.length:0,result=[];outer:while(++index>>1;callback(array[mid])1?arguments:arguments[0],index=-1,length=array?max(pluck(array,"length")):0,result=Array(length<0?0:length);while(++index2?createWrapper(func,17,slice(arguments,2),null,thisArg):createWrapper(func,1,null,null,thisArg)}function bindAll(object){var funcs=arguments.length>1?baseFlatten(arguments,true,false,1):functions(object),index=-1,length=funcs.length;while(++index2?createWrapper(key,19,slice(arguments,2),null,object):createWrapper(key,3,null,null,object)}function compose(){var funcs=arguments,length=funcs.length;while(length--){if(!isFunction(funcs[length])){throw new TypeError}}return function(){var args=arguments,length=funcs.length;while(length--){args=[funcs[length].apply(this,args)]}return args[0]}}function curry(func,arity){arity=typeof arity=="number"?arity:+arity||func.length;return createWrapper(func,4,null,null,null,arity)}function debounce(func,wait,options){var args,maxTimeoutId,result,stamp,thisArg,timeoutId,trailingCall,lastCalled=0,maxWait=false,trailing=true;if(!isFunction(func)){throw new TypeError}wait=nativeMax(0,wait)||0;if(options===true){var leading=true;trailing=false}else if(isObject(options)){leading=options.leading;maxWait="maxWait"in options&&(nativeMax(wait,options.maxWait)||0);trailing="trailing"in options?options.trailing:trailing}var delayed=function(){var remaining=wait-(now()-stamp);if(remaining<=0){if(maxTimeoutId){clearTimeout(maxTimeoutId); + +}var isCalled=trailingCall;maxTimeoutId=timeoutId=trailingCall=undefined;if(isCalled){lastCalled=now();result=func.apply(thisArg,args);if(!timeoutId&&!maxTimeoutId){args=thisArg=null}}}else{timeoutId=setTimeout(delayed,remaining)}};var maxDelayed=function(){if(timeoutId){clearTimeout(timeoutId)}maxTimeoutId=timeoutId=trailingCall=undefined;if(trailing||maxWait!==wait){lastCalled=now();result=func.apply(thisArg,args);if(!timeoutId&&!maxTimeoutId){args=thisArg=null}}};return function(){args=arguments;stamp=now();thisArg=this;trailingCall=trailing&&(timeoutId||!leading);if(maxWait===false){var leadingCall=leading&&!timeoutId}else{if(!maxTimeoutId&&!leading){lastCalled=stamp}var remaining=maxWait-(stamp-lastCalled),isCalled=remaining<=0;if(isCalled){if(maxTimeoutId){maxTimeoutId=clearTimeout(maxTimeoutId)}lastCalled=stamp;result=func.apply(thisArg,args)}else if(!maxTimeoutId){maxTimeoutId=setTimeout(maxDelayed,remaining)}}if(isCalled&&timeoutId){timeoutId=clearTimeout(timeoutId)}else if(!timeoutId&&wait!==maxWait){timeoutId=setTimeout(delayed,wait)}if(leadingCall){isCalled=true;result=func.apply(thisArg,args)}if(isCalled&&!timeoutId&&!maxTimeoutId){args=thisArg=null}return result}}function defer(func){if(!isFunction(func)){throw new TypeError}var args=slice(arguments,1);return setTimeout(function(){func.apply(undefined,args)},1)}function delay(func,wait){if(!isFunction(func)){throw new TypeError}var args=slice(arguments,2);return setTimeout(function(){func.apply(undefined,args)},wait)}function memoize(func,resolver){if(!isFunction(func)){throw new TypeError}var memoized=function(){var cache=memoized.cache,key=resolver?resolver.apply(this,arguments):keyPrefix+arguments[0];return hasOwnProperty.call(cache,key)?cache[key]:cache[key]=func.apply(this,arguments)};memoized.cache={};return memoized}function once(func){var ran,result;if(!isFunction(func)){throw new TypeError}return function(){if(ran){return result}ran=true;result=func.apply(this,arguments);func=null;return result}}function partial(func){return createWrapper(func,16,slice(arguments,1))}function partialRight(func){return createWrapper(func,32,null,slice(arguments,1))}function throttle(func,wait,options){var leading=true,trailing=true;if(!isFunction(func)){throw new TypeError}if(options===false){leading=false}else if(isObject(options)){leading="leading"in options?options.leading:leading;trailing="trailing"in options?options.trailing:trailing}debounceOptions.leading=leading;debounceOptions.maxWait=wait;debounceOptions.trailing=trailing;return debounce(func,wait,debounceOptions)}function wrap(value,wrapper){return createWrapper(wrapper,16,[value])}function constant(value){return function(){return value}}function createCallback(func,thisArg,argCount){var type=typeof func;if(func==null||type=="function"){return baseCreateCallback(func,thisArg,argCount)}if(type!="object"){return property(func)}var props=keys(func),key=props[0],a=func[key];if(props.length==1&&a===a&&!isObject(a)){return function(object){var b=object[key];return a===b&&(a!==0||1/a==1/b)}}return function(object){var length=props.length,result=false;while(length--){if(!(result=baseIsEqual(object[props[length]],func[props[length]],null,true))){break}}return result}}function escape(string){return string==null?"":String(string).replace(reUnescapedHtml,escapeHtmlChar)}function identity(value){return value}function mixin(object,source,options){var chain=true,methodNames=source&&functions(source);if(!source||!options&&!methodNames.length){if(options==null){options=source}ctor=lodashWrapper;source=object;object=lodash;methodNames=functions(source)}if(options===false){chain=false}else if(isObject(options)&&"chain"in options){chain=options.chain}var ctor=object,isFunc=isFunction(ctor);forEach(methodNames,function(methodName){var func=object[methodName]=source[methodName];if(isFunc){ctor.prototype[methodName]=function(){var chainAll=this.__chain__,value=this.__wrapped__,args=[value];push.apply(args,arguments);var result=func.apply(object,args);if(chain||chainAll){if(value===result&&isObject(result)){return this}result=new ctor(result);result.__chain__=chainAll}return result}}})}function noConflict(){context._=oldDash;return this}function noop(){}var now=isNative(now=Date.now)&&now||function(){return(new Date).getTime()};var parseInt=nativeParseInt(whitespace+"08")==8?nativeParseInt:function(value,radix){return nativeParseInt(isString(value)?value.replace(reLeadingSpacesAndZeros,""):value,radix||0)};function property(key){return function(object){return object[key]}}function random(min,max,floating){var noMin=min==null,noMax=max==null;if(floating==null){if(typeof min=="boolean"&&noMax){floating=min;min=1}else if(!noMax&&typeof max=="boolean"){floating=max;noMax=true}}if(noMin&&noMax){max=1}min=+min||0;if(noMax){max=min;min=0}else{max=+max||0}if(floating||min%1||max%1){var rand=nativeRandom();return nativeMin(min+rand*(max-min+parseFloat("1e-"+((rand+"").length-1))),max)}return baseRandom(min,max)}function result(object,key){if(object){var value=object[key];return isFunction(value)?object[key]():value}}function template(text,data,options){var settings=lodash.templateSettings;text=String(text||"");options=defaults({},options,settings);var imports=defaults({},options.imports,settings.imports),importsKeys=keys(imports),importsValues=values(imports);var isEvaluating,index=0,interpolate=options.interpolate||reNoMatch,source="__p += '";var reDelimiters=RegExp((options.escape||reNoMatch).source+"|"+interpolate.source+"|"+(interpolate===reInterpolate?reEsTemplate:reNoMatch).source+"|"+(options.evaluate||reNoMatch).source+"|$","g");text.replace(reDelimiters,function(match,escapeValue,interpolateValue,esTemplateValue,evaluateValue,offset){interpolateValue||(interpolateValue=esTemplateValue);source+=text.slice(index,offset).replace(reUnescapedString,escapeStringChar);if(escapeValue){source+="' +\n__e("+escapeValue+") +\n'"}if(evaluateValue){isEvaluating=true;source+="';\n"+evaluateValue+";\n__p += '"}if(interpolateValue){source+="' +\n((__t = ("+interpolateValue+")) == null ? '' : __t) +\n'"}index=offset+match.length;return match});source+="';\n";var variable=options.variable,hasVariable=variable;if(!hasVariable){variable="obj";source="with ("+variable+") {\n"+source+"\n}\n"}source=(isEvaluating?source.replace(reEmptyStringLeading,""):source).replace(reEmptyStringMiddle,"$1").replace(reEmptyStringTrailing,"$1;");source="function("+variable+") {\n"+(hasVariable?"":variable+" || ("+variable+" = {});\n")+"var __t, __p = '', __e = _.escape"+(isEvaluating?", __j = Array.prototype.join;\n"+"function print() { __p += __j.call(arguments, '') }\n":";\n")+source+"return __p\n}";var sourceURL="\n/*\n//# sourceURL="+(options.sourceURL||"/lodash/template/source["+templateCounter++ +"]")+"\n*/";try{var result=Function(importsKeys,"return "+source+sourceURL).apply(undefined,importsValues)}catch(e){e.source=source;throw e}if(data){return result(data)}result.source=source;return result}function times(n,callback,thisArg){n=(n=+n)>-1?n:0;var index=-1,result=Array(n);callback=baseCreateCallback(callback,thisArg,1);while(++index 0) { - // Take into account the position and width of the last stage's container - var existingStages = stageClusters(); + var existingStages = svgContainer + .selectAll("g.cluster") + .filter("[id*=\"" + VizConstants.stageClusterPrefix + "\"]"); if (!existingStages.empty()) { - var lastStage = existingStages[0].pop(); - var lastStageId = d3.select(lastStage).attr("id"); - var lastStageWidth = toFloat(d3.select("#" + lastStageId + " rect").attr("width")); - var lastStagePosition = getAbsolutePosition(lastStageId); + var lastStage = d3.select(existingStages[0].pop()); + var lastStageId = lastStage.attr("id"); + var lastStageWidth = toFloat(svgContainer + .select("#" + lastStageId) + .select("rect") + .attr("width")); + var lastStagePosition = getAbsolutePosition(lastStage); var offset = lastStagePosition.x + lastStageWidth + VizConstants.stageSep; container.attr("transform", "translate(" + offset + ", 0)"); } } + + // Actually render the stage renderDot(dot, container); + // If there are any incoming edges into this graph, keep track of them to render // them separately later. Note that we cannot draw them now because we need to // put these edges in a separate container that is on top of all stage graphs. @@ -221,15 +218,7 @@ function renderDagVizForJob(svgContainer) { }); }); - // Draw edges that cross stages - if (crossStageEdges.length > 0) { - var container = svgContainer.append("g").attr("id", "cross-stage-edges"); - for (var i = 0; i < crossStageEdges.length; i++) { - var fromRDDId = crossStageEdges[i][0]; - var toRDDId = crossStageEdges[i][1]; - connectRDDs(fromRDDId, toRDDId, container); - } - } + drawCrossStageEdges(crossStageEdges, svgContainer); } /* Render the dot file as an SVG in the given container. */ @@ -243,99 +232,156 @@ function renderDot(dot, container) { renderer(container, g); } -/* Style the visualization we just rendered. */ -function styleDagViz(forJob) { - graphContainer().selectAll("svg g.cluster rect") - .style("fill", "white") - .style("stroke", VizConstants.rddOperationColor) - .style("stroke-width", "4px") - .style("stroke-opacity", "0.5"); - graphContainer().selectAll("svg g.cluster text") - .attr("fill", VizConstants.clusterLabelColor) - .attr("font-size", "11px"); - graphContainer().selectAll("svg path") - .style("stroke", VizConstants.edgeColor) - .style("stroke-width", VizConstants.edgeWidth); - stageClusters() - .select("rect") - .style("stroke", VizConstants.stageColor) - .style("strokeWidth", "6px"); - - // Put an arrow at the end of every edge - // We need to do this because we manually render some edges ourselves - // For these edges, we borrow the arrow marker generated by dagre-d3 - var dagreD3Marker = graphContainer().select("svg g.edgePaths marker").node(); - graphContainer().select("svg") - .append(function() { return dagreD3Marker.cloneNode(true); }) - .attr("id", "marker-arrow") - .select("path") - .attr("fill", VizConstants.edgeColor) - .attr("strokeWidth", "0px"); - graphContainer().selectAll("svg g > path").attr("marker-end", "url(#marker-arrow)"); - graphContainer().selectAll("svg g.edgePaths def").remove(); // We no longer need these - - // Apply any job or stage specific styles +/* -------------------- * + * | Helper functions | * + * -------------------- */ + +// Helper d3 accessors +function graphContainer() { return d3.select("#dag-viz-graph"); } +function metadataContainer() { return d3.select("#dag-viz-metadata"); } + +/* + * Helper function to create draw a label for each cluster. + * + * We need to do this manually because dagre-d3 does not support labeling clusters. + * In general, the clustering support for dagre-d3 is quite limited at this point. + */ +function drawClusterLabels(svgContainer, forJob) { if (forJob) { - styleDagVizForJob(); + var clusterLabelSize = JobPageVizConstants.clusterLabelSize; + var stageClusterLabelSize = JobPageVizConstants.stageClusterLabelSize; } else { - styleDagVizForStage(); + var clusterLabelSize = StagePageVizConstants.clusterLabelSize; + var stageClusterLabelSize = StagePageVizConstants.stageClusterLabelSize; } + svgContainer.selectAll("g.cluster").each(function() { + var cluster = d3.select(this); + var isStage = cluster.attr("id").indexOf(VizConstants.stageClusterPrefix) > -1; + var labelSize = isStage ? stageClusterLabelSize : clusterLabelSize; + drawClusterLabel(cluster, labelSize); + }); } -/* Apply job-page-specific style to the visualization. */ -function styleDagVizForJob() { - graphContainer().selectAll("svg g.node circle") - .style("fill", VizConstants.rddColor); - // TODO: add a legend to explain what a highlighted dot means - graphContainer().selectAll("svg g.cached circle") - .style("fill", VizConstants.rddCachedColor); - graphContainer().selectAll("svg g#cross-stage-edges path") - .style("fill", "none"); +/* + * Helper function to draw a label for the given cluster element based on its name. + * + * In the process, we need to expand the bounding box to make room for the label. + * We need to do this because dagre-d3 did not take this into account when it first + * rendered the bounding boxes. Note that this means we need to adjust the view box + * of the SVG afterwards since we shifted a few boxes around. + */ +function drawClusterLabel(d3cluster, fontSize) { + var cluster = d3cluster; + var rect = d3cluster.select("rect"); + rect.attr("y", toFloat(rect.attr("y")) - fontSize); + rect.attr("height", toFloat(rect.attr("height")) + fontSize); + var labelX = toFloat(rect.attr("x")) + toFloat(rect.attr("width")) - fontSize / 2; + var labelY = toFloat(rect.attr("y")) + fontSize * 1.5; + var labelText = cluster.attr("name").replace(VizConstants.clusterPrefix, ""); + cluster.append("text") + .attr("x", labelX) + .attr("y", labelY) + .attr("text-anchor", "end") + .style("font-size", fontSize) + .text(labelText); } -/* Apply stage-page-specific style to the visualization. */ -function styleDagVizForStage() { - graphContainer().selectAll("svg g.node rect") - .style("fill", "none") - .style("stroke", VizConstants.rddColor) - .style("stroke-width", "2px") - .attr("rx", "5") // round corners - .attr("ry", "5"); - // TODO: add a legend to explain what a highlighted RDD means - graphContainer().selectAll("svg g.cached rect") - .style("stroke", VizConstants.rddCachedColor); - graphContainer().selectAll("svg g.node g.label text tspan") - .style("fill", VizConstants.rddColor); +/* + * Helper function to size the SVG appropriately such that all elements are displyed. + * This assumes that all outermost elements are clusters (rectangles). + */ +function resizeSvg(svg) { + var allClusters = svg.selectAll("g.cluster rect")[0]; + var startX = -VizConstants.svgMarginX + + toFloat(d3.min(allClusters, function(e) { + return getAbsolutePosition(d3.select(e)).x; + })); + var startY = -VizConstants.svgMarginY + + toFloat(d3.min(allClusters, function(e) { + return getAbsolutePosition(d3.select(e)).y; + })); + var endX = VizConstants.svgMarginX + + toFloat(d3.max(allClusters, function(e) { + var t = d3.select(e) + return getAbsolutePosition(t).x + toFloat(t.attr("width")); + })); + var endY = VizConstants.svgMarginY + + toFloat(d3.max(allClusters, function(e) { + var t = d3.select(e) + return getAbsolutePosition(t).y + toFloat(t.attr("height")); + })); + var width = endX - startX; + var height = endY - startY; + svg.attr("viewBox", startX + " " + startY + " " + width + " " + height) + .attr("width", width) + .attr("height", height); } /* - * (Job page only) Helper method to compute the absolute - * position of the group element identified by the given ID. + * (Job page only) Helper function to draw edges that cross stage boundaries. + * We need to do this manually because we render each stage separately in dagre-d3. */ -function getAbsolutePosition(groupId) { - var obj = d3.select("#" + groupId).filter("g"); - var _x = 0, _y = 0; +function drawCrossStageEdges(edges, svgContainer) { + if (edges.length == 0) { + return; + } + // Draw the paths first + var edgesContainer = svgContainer.append("g").attr("id", "cross-stage-edges"); + for (var i = 0; i < edges.length; i++) { + var fromRDDId = edges[i][0]; + var toRDDId = edges[i][1]; + connectRDDs(fromRDDId, toRDDId, edgesContainer, svgContainer); + } + // Now draw the arrows by borrowing the arrow marker generated by dagre-d3 + var dagreD3Marker = svgContainer.select("g.edgePaths marker").node(); + if (!dagreD3Marker.empty()) { + svgContainer + .append(function() { return dagreD3Marker.cloneNode(true); }) + .attr("id", "marker-arrow") + svgContainer.selectAll("g > path").attr("marker-end", "url(#marker-arrow)"); + svgContainer.selectAll("g.edgePaths def").remove(); // We no longer need these + } +} + +/* + * (Job page only) Helper function to compute the absolute + * position of the specified element in our graph. + */ +function getAbsolutePosition(d3selection) { + if (d3selection.empty()) { + throw "Attempted to get absolute position of an empty selection."; + } + var obj = d3selection; + var _x = toFloat(obj.attr("x")) || 0; + var _y = toFloat(obj.attr("y")) || 0; while (!obj.empty()) { var transformText = obj.attr("transform"); - var translate = d3.transform(transformText).translate - _x += translate[0]; - _y += translate[1]; - obj = d3.select(obj.node().parentNode).filter("g") + if (transformText) { + var translate = d3.transform(transformText).translate; + _x += toFloat(translate[0]); + _y += toFloat(translate[1]); + } + // Climb upwards to find how our parents are translated + obj = d3.select(obj.node().parentNode); + // Stop when we've reached the graph container itself + if (obj.node() == graphContainer().node()) { + break; + } } return { x: _x, y: _y }; } -/* (Job page only) Connect two RDD nodes with a curved edge. */ -function connectRDDs(fromRDDId, toRDDId, container) { +/* (Job page only) Helper function to connect two RDDs with a curved edge. */ +function connectRDDs(fromRDDId, toRDDId, edgesContainer, svgContainer) { var fromNodeId = VizConstants.nodePrefix + fromRDDId; - var toNodeId = VizConstants.nodePrefix + toRDDId - var fromPos = getAbsolutePosition(fromNodeId); - var toPos = getAbsolutePosition(toNodeId); + var toNodeId = VizConstants.nodePrefix + toRDDId; + var fromPos = getAbsolutePosition(svgContainer.select("#" + fromNodeId)); + var toPos = getAbsolutePosition(svgContainer.select("#" + toNodeId)); // On the job page, RDDs are rendered as dots (circles). When rendering the path, // we need to account for the radii of these circles. Otherwise the arrow heads // will bleed into the circle itself. - var delta = toFloat(graphContainer() + var delta = toFloat(svgContainer .select("g.node#" + toNodeId) .select("circle") .attr("r")); @@ -375,18 +421,15 @@ function connectRDDs(fromRDDId, toRDDId, container) { } var line = d3.svg.line().interpolate("basis"); - container.append("path").datum(points).attr("d", line); + edgesContainer.append("path").datum(points).attr("d", line); } -/* Helper d3 accessor to clusters that represent stages. */ -function stageClusters() { - return graphContainer().selectAll("g.cluster").filter(function() { - return d3.select(this).attr("id").indexOf(VizConstants.stageClusterPrefix) > -1; - }); -} - -/* Helper method to convert attributes to numeric values. */ +/* Helper function to convert attributes to numeric values. */ function toFloat(f) { - return parseFloat(f.replace(/px$/, "")); + if (f) { + return parseFloat(f.toString().replace(/px$/, "")); + } else { + return f; + } } diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala index 2f3fb181e4026..e2d03f8342315 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -156,13 +156,10 @@ private[spark] object UIUtils extends Logging { def commonHeaderNodes: Seq[Node] = { - - - - + + + + @@ -174,6 +171,7 @@ private[spark] object UIUtils extends Logging { } def vizHeaderNodes: Seq[Node] = { + @@ -358,7 +356,7 @@ private[spark] object UIUtils extends Logging {
      { graphs.map { g => - +## ElementwiseProduct + +ElementwiseProduct multiplies each input vector by a provided "weight" vector, using element-wise multiplication. In other words, it scales each column of the dataset by a scalar multiplier. This represents the [Hadamard product](https://en.wikipedia.org/wiki/Hadamard_product_%28matrices%29) between the input vector, `v` and transforming vector, `w`, to yield a result vector. + +`\[ \begin{pmatrix} +v_1 \\ +\vdots \\ +v_N +\end{pmatrix} \circ \begin{pmatrix} + w_1 \\ + \vdots \\ + w_N + \end{pmatrix} += \begin{pmatrix} + v_1 w_1 \\ + \vdots \\ + v_N w_N + \end{pmatrix} +\]` + +[`ElementwiseProduct`](api/scala/index.html#org.apache.spark.mllib.feature.ElementwiseProduct) has the following parameter in the constructor: + +* `w`: the transforming vector. + +`ElementwiseProduct` implements [`VectorTransformer`](api/scala/index.html#org.apache.spark.mllib.feature.VectorTransformer) which can apply the weighting on a `Vector` to produce a transformed `Vector` or on an `RDD[Vector]` to produce a transformed `RDD[Vector]`. + +### Example + +This example below demonstrates how to load a simple vectors file, extract a set of vectors, then transform those vectors using a transforming vector value. + + +
      +
      +{% highlight scala %} +import org.apache.spark.SparkContext._ +import org.apache.spark.mllib.feature.ElementwiseProduct +import org.apache.spark.mllib.linalg.Vectors + +// Load and parse the data: +val data = sc.textFile("data/mllib/kmeans_data.txt") +val parsedData = data.map(s => Vectors.dense(s.split(' ').map(_.toDouble))) + +val transformingVector = Vectors.dense(0.0, 1.0, 2.0) +val transformer = new ElementwiseProduct(transformingVector) + +// Batch transform and per-row transform give the same results: +val transformedData = transformer.transform(parsedData) +val transformedData2 = parsedData.map(x => transformer.transform(x)) + +{% endhighlight %} +
      +
      + + diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala new file mode 100644 index 0000000000000..f8b56293e3ccc --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.feature + +import org.apache.spark.annotation.AlphaComponent +import org.apache.spark.ml.UnaryTransformer +import org.apache.spark.ml.param.Param +import org.apache.spark.mllib.feature +import org.apache.spark.mllib.linalg.{Vector, VectorUDT} +import org.apache.spark.sql.types.DataType + +/** + * :: AlphaComponent :: + * Outputs the Hadamard product (i.e., the element-wise product) of each input vector with a + * provided "weight" vector. In other words, it scales each column of the dataset by a scalar + * multiplier. + */ +@AlphaComponent +class ElementwiseProduct extends UnaryTransformer[Vector, Vector, ElementwiseProduct] { + + /** + * the vector to multiply with input vectors + * @group param + */ + val scalingVec: Param[Vector] = new Param(this, "scalingVector", "vector for hadamard product") + + /** @group setParam */ + def setScalingVec(value: Vector): this.type = set(scalingVec, value) + + /** @group getParam */ + def getScalingVec: Vector = getOrDefault(scalingVec) + + override protected def createTransformFunc: Vector => Vector = { + require(params.contains(scalingVec), s"transformation requires a weight vector") + val elemScaler = new feature.ElementwiseProduct($(scalingVec)) + elemScaler.transform + } + + override protected def outputDataType: DataType = new VectorUDT() +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/ElementwiseProduct.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/ElementwiseProduct.scala new file mode 100644 index 0000000000000..b0985baf9b278 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/ElementwiseProduct.scala @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.feature + +import org.apache.spark.annotation.Experimental +import org.apache.spark.mllib.linalg._ + +/** + * :: Experimental :: + * Outputs the Hadamard product (i.e., the element-wise product) of each input vector with a + * provided "weight" vector. In other words, it scales each column of the dataset by a scalar + * multiplier. + * @param scalingVector The values used to scale the reference vector's individual components. + */ +@Experimental +class ElementwiseProduct(val scalingVector: Vector) extends VectorTransformer { + + /** + * Does the hadamard product transformation. + * + * @param vector vector to be transformed. + * @return transformed vector. + */ + override def transform(vector: Vector): Vector = { + require(vector.size == scalingVector.size, + s"vector sizes do not match: Expected ${scalingVector.size} but found ${vector.size}") + vector match { + case dv: DenseVector => + val values: Array[Double] = dv.values.clone() + val dim = scalingVector.size + var i = 0 + while (i < dim) { + values(i) *= scalingVector(i) + i += 1 + } + Vectors.dense(values) + case SparseVector(size, indices, vs) => + val values = vs.clone() + val dim = values.length + var i = 0 + while (i < dim) { + values(i) *= scalingVector(indices(i)) + i += 1 + } + Vectors.sparse(size, indices, values) + case v => throw new IllegalArgumentException("Does not support vector type " + v.getClass) + } + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/feature/ElementwiseProductSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/feature/ElementwiseProductSuite.scala new file mode 100644 index 0000000000000..f3a482abda873 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/feature/ElementwiseProductSuite.scala @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.feature + +import org.scalatest.FunSuite + +import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vectors} +import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.mllib.util.TestingUtils._ + +class ElementwiseProductSuite extends FunSuite with MLlibTestSparkContext { + + test("elementwise (hadamard) product should properly apply vector to dense data set") { + val denseData = Array( + Vectors.dense(1.0, 4.0, 1.9, -9.0) + ) + val scalingVec = Vectors.dense(2.0, 0.5, 0.0, 0.25) + val transformer = new ElementwiseProduct(scalingVec) + val transformedData = transformer.transform(sc.makeRDD(denseData)) + val transformedVecs = transformedData.collect() + val transformedVec = transformedVecs(0) + val expectedVec = Vectors.dense(2.0, 2.0, 0.0, -2.25) + assert(transformedVec ~== expectedVec absTol 1E-5, + s"Expected transformed vector $expectedVec but found $transformedVec") + } + + test("elementwise (hadamard) product should properly apply vector to sparse data set") { + val sparseData = Array( + Vectors.sparse(3, Seq((1, -1.0), (2, -3.0))) + ) + val dataRDD = sc.parallelize(sparseData, 3) + val scalingVec = Vectors.dense(1.0, 0.0, 0.5) + val transformer = new ElementwiseProduct(scalingVec) + val data2 = sparseData.map(transformer.transform) + val data2RDD = transformer.transform(dataRDD) + + assert((sparseData, data2, data2RDD.collect()).zipped.forall { + case (v1: DenseVector, v2: DenseVector, v3: DenseVector) => true + case (v1: SparseVector, v2: SparseVector, v3: SparseVector) => true + case _ => false + }, "The vector type should be preserved after hadamard product") + + assert((data2, data2RDD.collect()).zipped.forall((v1, v2) => v1 ~== v2 absTol 1E-5)) + assert(data2(0) ~== Vectors.sparse(3, Seq((1, 0.0), (2, -1.5))) absTol 1E-5) + } +} From e43803b8f477b2c8d28836ac163cb54328d13f1a Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Thu, 7 May 2015 15:45:37 -0700 Subject: [PATCH 45/75] [SPARK-6948] [MLLIB] compress vectors in VectorAssembler The compression is based on storage. brkyvz Author: Xiangrui Meng Closes #5985 from mengxr/SPARK-6948 and squashes the following commits: df56a00 [Xiangrui Meng] update python tests 6d90d45 [Xiangrui Meng] compress vectors in VectorAssembler --- .../org/apache/spark/ml/feature/VectorAssembler.scala | 2 +- .../apache/spark/ml/feature/VectorAssemblerSuite.scala | 10 +++++++++- python/pyspark/ml/feature.py | 6 +++--- 3 files changed, 13 insertions(+), 5 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorAssembler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorAssembler.scala index b5a69cee6daf3..796758a70ef18 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorAssembler.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorAssembler.scala @@ -102,6 +102,6 @@ object VectorAssembler { case o => throw new SparkException(s"$o of type ${o.getClass.getName} is not supported.") } - Vectors.sparse(cur, indices.result(), values.result()) + Vectors.sparse(cur, indices.result(), values.result()).compressed } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/VectorAssemblerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/VectorAssemblerSuite.scala index 57d0278e03639..0db27607bc274 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/VectorAssemblerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/VectorAssemblerSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.ml.feature import org.scalatest.FunSuite import org.apache.spark.SparkException -import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, Vectors} import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.sql.{Row, SQLContext} @@ -48,6 +48,14 @@ class VectorAssemblerSuite extends FunSuite with MLlibTestSparkContext { } } + test("assemble should compress vectors") { + import org.apache.spark.ml.feature.VectorAssembler.assemble + val v1 = assemble(0.0, 0.0, 0.0, Vectors.dense(4.0)) + assert(v1.isInstanceOf[SparseVector]) + val v2 = assemble(1.0, 2.0, 3.0, Vectors.sparse(1, Array(0), Array(4.0))) + assert(v2.isInstanceOf[DenseVector]) + } + test("VectorAssembler") { val df = sqlContext.createDataFrame(Seq( (0, 0.0, Vectors.dense(1.0, 2.0), "a", Vectors.sparse(2, Array(1), Array(3.0)), 10L) diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index 8a0fdddd2d9b5..705a368192c24 100644 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -121,12 +121,12 @@ class VectorAssembler(JavaTransformer, HasInputCols, HasOutputCol): >>> df = sc.parallelize([Row(a=1, b=0, c=3)]).toDF() >>> vecAssembler = VectorAssembler(inputCols=["a", "b", "c"], outputCol="features") >>> vecAssembler.transform(df).head().features - SparseVector(3, {0: 1.0, 2: 3.0}) + DenseVector([1.0, 0.0, 3.0]) >>> vecAssembler.setParams(outputCol="freqs").transform(df).head().freqs - SparseVector(3, {0: 1.0, 2: 3.0}) + DenseVector([1.0, 0.0, 3.0]) >>> params = {vecAssembler.inputCols: ["b", "a"], vecAssembler.outputCol: "vector"} >>> vecAssembler.transform(df, params).head().vector - SparseVector(2, {1: 1.0}) + DenseVector([0.0, 1.0]) """ _java_class = "org.apache.spark.ml.feature.VectorAssembler" From 97d1182af63d55abab44521171652c81c56c6af6 Mon Sep 17 00:00:00 2001 From: scwf Date: Thu, 7 May 2015 16:21:24 -0700 Subject: [PATCH 46/75] [SQL] [MINOR] make star and multialias extend NamedExpression `Star` and `MultiAlias` just used in `analyzer` and them will be substituted after analyze, So just like `Alias` they do not need extend `Attribute` Author: scwf Closes #5928 from scwf/attribute and squashes the following commits: 73a0560 [scwf] star and multialias do not need extend attribute --- .../sql/catalyst/analysis/unresolved.scala | 20 +++++-------------- .../expressions/namedExpressions.scala | 2 +- 2 files changed, 6 insertions(+), 16 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala index 3f567e3e8b2a6..eb736ac3290e8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala @@ -95,7 +95,7 @@ case class UnresolvedFunction(name: String, children: Seq[Expression]) extends E * Represents all of the input attributes to a given relational operator, for example in * "SELECT * FROM ...". A [[Star]] gets automatically expanded during analysis. */ -trait Star extends Attribute with trees.LeafNode[Expression] { +trait Star extends NamedExpression with trees.LeafNode[Expression] { self: Product => override def name: String = throw new UnresolvedException(this, "name") @@ -103,13 +103,9 @@ trait Star extends Attribute with trees.LeafNode[Expression] { override def dataType: DataType = throw new UnresolvedException(this, "dataType") override def nullable: Boolean = throw new UnresolvedException(this, "nullable") override def qualifiers: Seq[String] = throw new UnresolvedException(this, "qualifiers") + override def toAttribute: Attribute = throw new UnresolvedException(this, "toAttribute") override lazy val resolved = false - override def newInstance(): Star = this - override def withNullability(newNullability: Boolean): Star = this - override def withQualifiers(newQualifiers: Seq[String]): Star = this - override def withName(newName: String): Star = this - // Star gets expanded at runtime so we never evaluate a Star. override def eval(input: Row = null): EvaluatedType = throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}") @@ -154,7 +150,7 @@ case class UnresolvedStar(table: Option[String]) extends Star { * @param names the names to be associated with each output of computing [[child]]. */ case class MultiAlias(child: Expression, names: Seq[String]) - extends Attribute with trees.UnaryNode[Expression] { + extends NamedExpression with trees.UnaryNode[Expression] { override def name: String = throw new UnresolvedException(this, "name") @@ -166,15 +162,9 @@ case class MultiAlias(child: Expression, names: Seq[String]) override def qualifiers: Seq[String] = throw new UnresolvedException(this, "qualifiers") - override lazy val resolved = false - - override def newInstance(): MultiAlias = this + override def toAttribute: Attribute = throw new UnresolvedException(this, "toAttribute") - override def withNullability(newNullability: Boolean): MultiAlias = this - - override def withQualifiers(newQualifiers: Seq[String]): MultiAlias = this - - override def withName(newName: String): MultiAlias = this + override lazy val resolved = false override def eval(input: Row = null): EvaluatedType = throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala index 57ace2a14f77c..a9170589f8c6c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala @@ -86,7 +86,7 @@ abstract class Attribute extends NamedExpression { def withQualifiers(newQualifiers: Seq[String]): Attribute def withName(newName: String): Attribute - def toAttribute: Attribute = this + override def toAttribute: Attribute = this def newInstance(): Attribute } From ea3077f19c18b5556a3632b36771aeb153746ff5 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Thu, 7 May 2015 16:22:45 -0700 Subject: [PATCH 47/75] [SPARK-7277] [SQL] Throw exception if the property mapred.reduce.tasks is set to -1 JIRA: https://issues.apache.org/jira/browse/SPARK-7277 As automatically determining the number of reducers is not supported (`mapred.reduce.tasks` is set to `-1`), we should throw exception to users. Author: Liang-Chi Hsieh Closes #5811 from viirya/no_neg_reduce_tasks and squashes the following commits: e518f96 [Liang-Chi Hsieh] Consider other wrong setting values. fd9c817 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into no_neg_reduce_tasks 4ede705 [Liang-Chi Hsieh] Throw exception instead of warning message. 68a1c70 [Liang-Chi Hsieh] Show warning message if mapred.reduce.tasks is set to -1. --- .../org/apache/spark/sql/execution/commands.scala | 10 ++++++++-- .../scala/org/apache/spark/sql/SQLQuerySuite.scala | 10 ++++++++++ 2 files changed, 18 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala index 98df5bef34efa..65687db4e6235 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala @@ -84,8 +84,14 @@ case class SetCommand( logWarning( s"Property ${SQLConf.Deprecated.MAPRED_REDUCE_TASKS} is deprecated, " + s"automatically converted to ${SQLConf.SHUFFLE_PARTITIONS} instead.") - sqlContext.setConf(SQLConf.SHUFFLE_PARTITIONS, value) - Seq(Row(s"${SQLConf.SHUFFLE_PARTITIONS}=$value")) + if (value.toInt < 1) { + val msg = s"Setting negative ${SQLConf.Deprecated.MAPRED_REDUCE_TASKS} for automatically " + + "determining the number of reducers is not supported." + throw new IllegalArgumentException(msg) + } else { + sqlContext.setConf(SQLConf.SHUFFLE_PARTITIONS, value) + Seq(Row(s"${SQLConf.SHUFFLE_PARTITIONS}=$value")) + } // Configures a single property. case Some((key, Some(value))) => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 208cec6a32d4d..77be3b8b206c0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -871,6 +871,16 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { conf.clear() } + test("SET commands with illegal or inappropriate argument") { + conf.clear() + // Set negative mapred.reduce.tasks for automatically determing + // the number of reducers is not supported + intercept[IllegalArgumentException](sql(s"SET mapred.reduce.tasks=-1")) + intercept[IllegalArgumentException](sql(s"SET mapred.reduce.tasks=-01")) + intercept[IllegalArgumentException](sql(s"SET mapred.reduce.tasks=-2")) + conf.clear() + } + test("apply schema") { val schema1 = StructType( StructField("f1", IntegerType, false) :: From 937ba798c56770ec54276b9259e47ae65ee93967 Mon Sep 17 00:00:00 2001 From: Iulian Dragos Date: Thu, 7 May 2015 16:24:11 -0700 Subject: [PATCH 48/75] [SPARK-5281] [SQL] Registering table on RDD is giving MissingRequirementError Go through the context classloader when reflecting on user types in ScalaReflection. Replaced calls to `typeOf` with `typeTag[T].in(mirror)`. The convenience method assumes all types can be found in the classloader that loaded scala-reflect (the primordial classloader). This assumption is not valid in all contexts (sbt console, Eclipse launchers). Fixed SPARK-5281 Author: Iulian Dragos Closes #5981 from dragos/issue/mirrors-missing-requirement-error and squashes the following commits: d103e70 [Iulian Dragos] Go through the context classloader when reflecting on user types in ScalaReflection --- .../spark/sql/catalyst/ScalaReflection.scala | 57 ++++++++++++------- 1 file changed, 37 insertions(+), 20 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala index c52965507c715..6998cc8d9666d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala @@ -27,6 +27,7 @@ import org.apache.spark.sql.types._ */ object ScalaReflection extends ScalaReflection { val universe: scala.reflect.runtime.universe.type = scala.reflect.runtime.universe + val mirror: universe.Mirror = universe.runtimeMirror(Thread.currentThread().getContextClassLoader) } /** @@ -36,6 +37,9 @@ trait ScalaReflection { /** The universe we work in (runtime or macro) */ val universe: scala.reflect.api.Universe + /** The mirror used to access types in the universe */ + val mirror: universe.Mirror + import universe._ // The Predef.Map is scala.collection.immutable.Map. @@ -52,7 +56,19 @@ trait ScalaReflection { /** Returns a catalyst DataType and its nullability for the given Scala Type using reflection. */ def schemaFor[T: TypeTag]: Schema = - ScalaReflectionLock.synchronized { schemaFor(typeOf[T]) } + ScalaReflectionLock.synchronized { schemaFor(localTypeOf[T]) } + + /** + * Return the Scala Type for `T` in the current classloader mirror. + * + * Use this method instead of the convenience method `universe.typeOf`, which + * assumes that all types can be found in the classloader that loaded scala-reflect classes. + * That's not necessarily the case when running using Eclipse launchers or even + * Sbt console or test (without `fork := true`). + * + * @see SPARK-5281 + */ + private def localTypeOf[T: TypeTag]: `Type` = typeTag[T].in(mirror).tpe /** Returns a catalyst DataType and its nullability for the given Scala Type using reflection. */ def schemaFor(tpe: `Type`): Schema = ScalaReflectionLock.synchronized { @@ -67,25 +83,25 @@ trait ScalaReflection { val udt = Utils.classForName(className) .getAnnotation(classOf[SQLUserDefinedType]).udt().newInstance() Schema(udt, nullable = true) - case t if t <:< typeOf[Option[_]] => + case t if t <:< localTypeOf[Option[_]] => val TypeRef(_, _, Seq(optType)) = t Schema(schemaFor(optType).dataType, nullable = true) // Need to decide if we actually need a special type here. - case t if t <:< typeOf[Array[Byte]] => Schema(BinaryType, nullable = true) - case t if t <:< typeOf[Array[_]] => + case t if t <:< localTypeOf[Array[Byte]] => Schema(BinaryType, nullable = true) + case t if t <:< localTypeOf[Array[_]] => val TypeRef(_, _, Seq(elementType)) = t val Schema(dataType, nullable) = schemaFor(elementType) Schema(ArrayType(dataType, containsNull = nullable), nullable = true) - case t if t <:< typeOf[Seq[_]] => + case t if t <:< localTypeOf[Seq[_]] => val TypeRef(_, _, Seq(elementType)) = t val Schema(dataType, nullable) = schemaFor(elementType) Schema(ArrayType(dataType, containsNull = nullable), nullable = true) - case t if t <:< typeOf[Map[_, _]] => + case t if t <:< localTypeOf[Map[_, _]] => val TypeRef(_, _, Seq(keyType, valueType)) = t val Schema(valueDataType, valueNullable) = schemaFor(valueType) Schema(MapType(schemaFor(keyType).dataType, valueDataType, valueContainsNull = valueNullable), nullable = true) - case t if t <:< typeOf[Product] => + case t if t <:< localTypeOf[Product] => val formalTypeArgs = t.typeSymbol.asClass.typeParams val TypeRef(_, _, actualTypeArgs) = t val constructorSymbol = t.member(nme.CONSTRUCTOR) @@ -107,19 +123,20 @@ trait ScalaReflection { schemaFor(p.typeSignature.substituteTypes(formalTypeArgs, actualTypeArgs)) StructField(p.name.toString, dataType, nullable) }), nullable = true) - case t if t <:< typeOf[String] => Schema(StringType, nullable = true) - case t if t <:< typeOf[java.sql.Timestamp] => Schema(TimestampType, nullable = true) - case t if t <:< typeOf[java.sql.Date] => Schema(DateType, nullable = true) - case t if t <:< typeOf[BigDecimal] => Schema(DecimalType.Unlimited, nullable = true) - case t if t <:< typeOf[java.math.BigDecimal] => Schema(DecimalType.Unlimited, nullable = true) - case t if t <:< typeOf[Decimal] => Schema(DecimalType.Unlimited, nullable = true) - case t if t <:< typeOf[java.lang.Integer] => Schema(IntegerType, nullable = true) - case t if t <:< typeOf[java.lang.Long] => Schema(LongType, nullable = true) - case t if t <:< typeOf[java.lang.Double] => Schema(DoubleType, nullable = true) - case t if t <:< typeOf[java.lang.Float] => Schema(FloatType, nullable = true) - case t if t <:< typeOf[java.lang.Short] => Schema(ShortType, nullable = true) - case t if t <:< typeOf[java.lang.Byte] => Schema(ByteType, nullable = true) - case t if t <:< typeOf[java.lang.Boolean] => Schema(BooleanType, nullable = true) + case t if t <:< localTypeOf[String] => Schema(StringType, nullable = true) + case t if t <:< localTypeOf[java.sql.Timestamp] => Schema(TimestampType, nullable = true) + case t if t <:< localTypeOf[java.sql.Date] => Schema(DateType, nullable = true) + case t if t <:< localTypeOf[BigDecimal] => Schema(DecimalType.Unlimited, nullable = true) + case t if t <:< localTypeOf[java.math.BigDecimal] => + Schema(DecimalType.Unlimited, nullable = true) + case t if t <:< localTypeOf[Decimal] => Schema(DecimalType.Unlimited, nullable = true) + case t if t <:< localTypeOf[java.lang.Integer] => Schema(IntegerType, nullable = true) + case t if t <:< localTypeOf[java.lang.Long] => Schema(LongType, nullable = true) + case t if t <:< localTypeOf[java.lang.Double] => Schema(DoubleType, nullable = true) + case t if t <:< localTypeOf[java.lang.Float] => Schema(FloatType, nullable = true) + case t if t <:< localTypeOf[java.lang.Short] => Schema(ShortType, nullable = true) + case t if t <:< localTypeOf[java.lang.Byte] => Schema(ByteType, nullable = true) + case t if t <:< localTypeOf[java.lang.Boolean] => Schema(BooleanType, nullable = true) case t if t <:< definitions.IntTpe => Schema(IntegerType, nullable = false) case t if t <:< definitions.LongTpe => Schema(LongType, nullable = false) case t if t <:< definitions.DoubleTpe => Schema(DoubleType, nullable = false) From 35f0173b8f67e2e506fc4575be6430cfb66e2238 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Thu, 7 May 2015 16:26:49 -0700 Subject: [PATCH 49/75] [SPARK-2155] [SQL] [WHEN D THEN E] [ELSE F] add CaseKeyWhen for "CASE a WHEN b THEN c * END" Avoid translating to CaseWhen and evaluate the key expression many times. Author: Wenchen Fan Closes #5979 from cloud-fan/condition and squashes the following commits: 3ce54e1 [Wenchen Fan] add CaseKeyWhen --- .../apache/spark/sql/catalyst/SqlParser.scala | 10 +- .../catalyst/analysis/HiveTypeCoercion.scala | 43 +++--- .../sql/catalyst/expressions/Expression.scala | 2 +- .../sql/catalyst/expressions/predicates.scala | 135 ++++++++++++------ .../ExpressionEvaluationSuite.scala | 26 ++++ .../spark/sql/DataFrameNaFunctions.scala | 9 +- .../org/apache/spark/sql/hive/HiveQl.scala | 12 +- .../sql/hive/execution/SQLQuerySuite.scala | 7 + 8 files changed, 159 insertions(+), 85 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala index 1d3a2dc0d9bb0..b06bfb2ce8c8e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -296,13 +296,13 @@ class SqlParser extends AbstractSparkSQLParser with DataTypeParser { | LOWER ~ "(" ~> expression <~ ")" ^^ { case exp => Lower(exp) } | IF ~ "(" ~> expression ~ ("," ~> expression) ~ ("," ~> expression) <~ ")" ^^ { case c ~ t ~ f => If(c, t, f) } - | CASE ~> expression.? ~ (WHEN ~> expression ~ (THEN ~> expression)).* ~ + | CASE ~> expression.? ~ rep1(WHEN ~> expression ~ (THEN ~> expression)) ~ (ELSE ~> expression).? <~ END ^^ { case casePart ~ altPart ~ elsePart => - val altExprs = altPart.flatMap { case whenExpr ~ thenExpr => - Seq(casePart.fold(whenExpr)(EqualTo(_, whenExpr)), thenExpr) - } - CaseWhen(altExprs ++ elsePart.toList) + val branches = altPart.flatMap { case whenExpr ~ thenExpr => + Seq(whenExpr, thenExpr) + } ++ elsePart + casePart.map(CaseKeyWhen(_, branches)).getOrElse(CaseWhen(branches)) } | (SUBSTR | SUBSTRING) ~ "(" ~> expression ~ ("," ~> expression) <~ ")" ^^ { case s ~ p => Substring(s, p, Literal(Integer.MAX_VALUE)) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala index 873c75c525c3b..168a4e30eab86 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala @@ -631,31 +631,24 @@ trait HiveTypeCoercion { import HiveTypeCoercion._ def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { - case cw @ CaseWhen(branches) if !cw.resolved && !branches.exists(!_.resolved) => - val valueTypes = branches.sliding(2, 2).map { - case Seq(_, value) => value.dataType - case Seq(elseVal) => elseVal.dataType - }.toSeq - - logDebug(s"Input values for null casting ${valueTypes.mkString(",")}") - - if (valueTypes.distinct.size > 1) { - val commonType = valueTypes.reduce { (v1, v2) => - findTightestCommonType(v1, v2) - .getOrElse(sys.error( - s"Types in CASE WHEN must be the same or coercible to a common type: $v1 != $v2")) - } - val transformedBranches = branches.sliding(2, 2).map { - case Seq(cond, value) if value.dataType != commonType => - Seq(cond, Cast(value, commonType)) - case Seq(elseVal) if elseVal.dataType != commonType => - Seq(Cast(elseVal, commonType)) - case s => s - }.reduce(_ ++ _) - CaseWhen(transformedBranches) - } else { - // Types match up. Hopefully some other rule fixes whatever is wrong with resolution. - cw + case cw: CaseWhenLike if !cw.resolved && cw.childrenResolved && !cw.valueTypesEqual => + logDebug(s"Input values for null casting ${cw.valueTypes.mkString(",")}") + val commonType = cw.valueTypes.reduce { (v1, v2) => + findTightestCommonType(v1, v2).getOrElse(sys.error( + s"Types in CASE WHEN must be the same or coercible to a common type: $v1 != $v2")) + } + val transformedBranches = cw.branches.sliding(2, 2).map { + case Seq(when, value) if value.dataType != commonType => + Seq(when, Cast(value, commonType)) + case Seq(elseVal) if elseVal.dataType != commonType => + Seq(Cast(elseVal, commonType)) + case s => s + }.reduce(_ ++ _) + cw match { + case _: CaseWhen => + CaseWhen(transformedBranches) + case CaseKeyWhen(key, _) => + CaseKeyWhen(key, transformedBranches) } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala index 4fd1bc4dd642d..0837a3179d897 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala @@ -64,7 +64,7 @@ abstract class Expression extends TreeNode[Expression] { * Returns true if all the children of this expression have been resolved to a specific schema * and false if any still contains any unresolved placeholders. */ - def childrenResolved: Boolean = !children.exists(!_.resolved) + def childrenResolved: Boolean = children.forall(_.resolved) /** * Returns a string representation of this expression that does not have developer centric diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala index 26c38c56c04f5..50b0f3ee5f93f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala @@ -353,79 +353,134 @@ case class If(predicate: Expression, trueValue: Expression, falseValue: Expressi override def toString: String = s"if ($predicate) $trueValue else $falseValue" } +trait CaseWhenLike extends Expression { + self: Product => + + type EvaluatedType = Any + + // Note that `branches` are considered in consecutive pairs (cond, val), and the optional last + // element is the value for the default catch-all case (if provided). + // Hence, `branches` consists of at least two elements, and can have an odd or even length. + def branches: Seq[Expression] + + @transient lazy val whenList = + branches.sliding(2, 2).collect { case Seq(whenExpr, _) => whenExpr }.toSeq + @transient lazy val thenList = + branches.sliding(2, 2).collect { case Seq(_, thenExpr) => thenExpr }.toSeq + val elseValue = if (branches.length % 2 == 0) None else Option(branches.last) + + // both then and else val should be considered. + def valueTypes: Seq[DataType] = (thenList ++ elseValue).map(_.dataType) + def valueTypesEqual: Boolean = valueTypes.distinct.size <= 1 + + override def dataType: DataType = { + if (!resolved) { + throw new UnresolvedException(this, "cannot resolve due to differing types in some branches") + } + valueTypes.head + } + + override def nullable: Boolean = { + // If no value is nullable and no elseValue is provided, the whole statement defaults to null. + thenList.exists(_.nullable) || (elseValue.map(_.nullable).getOrElse(true)) + } +} + // scalastyle:off /** * Case statements of the form "CASE WHEN a THEN b [WHEN c THEN d]* [ELSE e] END". * Refer to this link for the corresponding semantics: * https://cwiki.apache.org/confluence/display/Hive/LanguageManual+UDF#LanguageManualUDF-ConditionalFunctions - * - * The other form of case statements "CASE a WHEN b THEN c [WHEN d THEN e]* [ELSE f] END" gets - * translated to this form at parsing time. Namely, such a statement gets translated to - * "CASE WHEN a=b THEN c [WHEN a=d THEN e]* [ELSE f] END". - * - * Note that `branches` are considered in consecutive pairs (cond, val), and the optional last - * element is the value for the default catch-all case (if provided). Hence, `branches` consists of - * at least two elements, and can have an odd or even length. */ // scalastyle:on -case class CaseWhen(branches: Seq[Expression]) extends Expression { - type EvaluatedType = Any +case class CaseWhen(branches: Seq[Expression]) extends CaseWhenLike { + + // Use private[this] Array to speed up evaluation. + @transient private[this] lazy val branchesArr = branches.toArray override def children: Seq[Expression] = branches - override def dataType: DataType = { - if (!resolved) { - throw new UnresolvedException(this, "cannot resolve due to differing types in some branches") + override lazy val resolved: Boolean = + childrenResolved && + whenList.forall(_.dataType == BooleanType) && + valueTypesEqual + + /** Written in imperative fashion for performance considerations. */ + override def eval(input: Row): Any = { + val len = branchesArr.length + var i = 0 + // If all branches fail and an elseVal is not provided, the whole statement + // defaults to null, according to Hive's semantics. + while (i < len - 1) { + if (branchesArr(i).eval(input) == true) { + return branchesArr(i + 1).eval(input) + } + i += 2 + } + var res: Any = null + if (i == len - 1) { + res = branchesArr(i).eval(input) } - branches(1).dataType + return res } + override def toString: String = { + "CASE" + branches.sliding(2, 2).map { + case Seq(cond, value) => s" WHEN $cond THEN $value" + case Seq(elseValue) => s" ELSE $elseValue" + }.mkString + } +} + +// scalastyle:off +/** + * Case statements of the form "CASE a WHEN b THEN c [WHEN d THEN e]* [ELSE f] END". + * Refer to this link for the corresponding semantics: + * https://cwiki.apache.org/confluence/display/Hive/LanguageManual+UDF#LanguageManualUDF-ConditionalFunctions + */ +// scalastyle:on +case class CaseKeyWhen(key: Expression, branches: Seq[Expression]) extends CaseWhenLike { + + // Use private[this] Array to speed up evaluation. @transient private[this] lazy val branchesArr = branches.toArray - @transient private[this] lazy val predicates = - branches.sliding(2, 2).collect { case Seq(cond, _) => cond }.toSeq - @transient private[this] lazy val values = - branches.sliding(2, 2).collect { case Seq(_, value) => value }.toSeq - @transient private[this] lazy val elseValue = - if (branches.length % 2 == 0) None else Option(branches.last) - override def nullable: Boolean = { - // If no value is nullable and no elseValue is provided, the whole statement defaults to null. - values.exists(_.nullable) || (elseValue.map(_.nullable).getOrElse(true)) - } + override def children: Seq[Expression] = key +: branches - override lazy val resolved: Boolean = { - if (!childrenResolved) { - false - } else { - val allCondBooleans = predicates.forall(_.dataType == BooleanType) - // both then and else val should be considered. - val dataTypesEqual = (values ++ elseValue).map(_.dataType).distinct.size <= 1 - allCondBooleans && dataTypesEqual - } - } + override lazy val resolved: Boolean = + childrenResolved && valueTypesEqual /** Written in imperative fashion for performance considerations. */ override def eval(input: Row): Any = { + val evaluatedKey = key.eval(input) val len = branchesArr.length var i = 0 // If all branches fail and an elseVal is not provided, the whole statement // defaults to null, according to Hive's semantics. - var res: Any = null while (i < len - 1) { - if (branchesArr(i).eval(input) == true) { - res = branchesArr(i + 1).eval(input) - return res + if (equalNullSafe(evaluatedKey, branchesArr(i).eval(input))) { + return branchesArr(i + 1).eval(input) } i += 2 } + var res: Any = null if (i == len - 1) { res = branchesArr(i).eval(input) } - res + return res + } + + private def equalNullSafe(l: Any, r: Any) = { + if (l == null && r == null) { + true + } else if (l == null || r == null) { + false + } else { + l == r + } } override def toString: String = { - "CASE" + branches.sliding(2, 2).map { + s"CASE $key" + branches.sliding(2, 2).map { case Seq(cond, value) => s" WHEN $cond THEN $value" case Seq(elseValue) => s" ELSE $elseValue" }.mkString diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala index faaa55aa5e6f1..88d36d153ceea 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala @@ -850,6 +850,32 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { assert(CaseWhen(Seq(c2, c4_notNull, c3, c5)).nullable === true) } + test("case key when") { + val row = create_row(null, 1, 2, "a", "b", "c") + val c1 = 'a.int.at(0) + val c2 = 'a.int.at(1) + val c3 = 'a.int.at(2) + val c4 = 'a.string.at(3) + val c5 = 'a.string.at(4) + val c6 = 'a.string.at(5) + + val literalNull = Literal.create(null, BooleanType) + val literalInt = Literal(1) + val literalString = Literal("a") + + checkEvaluation(CaseKeyWhen(c1, Seq(c2, c4, c5)), "b", row) + checkEvaluation(CaseKeyWhen(c1, Seq(c2, c4, literalNull, c5, c6)), "b", row) + checkEvaluation(CaseKeyWhen(c2, Seq(literalInt, c4, c5)), "a", row) + checkEvaluation(CaseKeyWhen(c2, Seq(c1, c4, c5)), "b", row) + checkEvaluation(CaseKeyWhen(c4, Seq(literalString, c2, c3)), 1, row) + checkEvaluation(CaseKeyWhen(c4, Seq(c1, c3, c5, c2, Literal(3))), 3, row) + + checkEvaluation(CaseKeyWhen(literalInt, Seq(c2, c4, c5)), "a", row) + checkEvaluation(CaseKeyWhen(literalString, Seq(c5, c2, c4, c3)), 2, row) + checkEvaluation(CaseKeyWhen(literalInt, Seq(c5, c2, c4, c3)), null, row) + checkEvaluation(CaseKeyWhen(literalNull, Seq(c5, c2, c1, c3)), 2, row) + } + test("complex type") { val row = create_row( "^Ba*n", // 0 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala index 481ed4924857e..4a54120ba86f6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala @@ -357,11 +357,12 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) { * TODO: This can be optimized to use broadcast join when replacementMap is large. */ private def replaceCol(col: StructField, replacementMap: Map[_, _]): Column = { - val branches: Seq[Expression] = replacementMap.flatMap { case (source, target) => - df.col(col.name).equalTo(lit(source).cast(col.dataType)).expr :: - lit(target).cast(col.dataType).expr :: Nil + val keyExpr = df.col(col.name).expr + def buildExpr(v: Any) = Cast(Literal(v), keyExpr.dataType) + val branches = replacementMap.flatMap { case (source, target) => + Seq(buildExpr(source), buildExpr(target)) }.toSeq - new Column(CaseWhen(branches ++ Seq(df.col(col.name).expr))).as(col.name) + new Column(CaseKeyWhen(keyExpr, branches :+ keyExpr)).as(col.name) } private def convertToDouble(v: Any): Double = v match { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index 4e51473979a2a..6176aee25e7a4 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -1246,16 +1246,8 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C case Token("TOK_FUNCTION", Token(WHEN(), Nil) :: branches) => CaseWhen(branches.map(nodeToExpr)) case Token("TOK_FUNCTION", Token(CASE(), Nil) :: branches) => - val transformed = branches.drop(1).sliding(2, 2).map { - case Seq(condVal, value) => - // FIXME (SPARK-2155): the key will get evaluated for multiple times in CaseWhen's eval(). - // Hence effectful / non-deterministic key expressions are *not* supported at the moment. - // We should consider adding new Expressions to get around this. - Seq(EqualTo(nodeToExpr(branches(0)), nodeToExpr(condVal)), - nodeToExpr(value)) - case Seq(elseVal) => Seq(nodeToExpr(elseVal)) - }.toSeq.reduce(_ ++ _) - CaseWhen(transformed) + val keyExpr = nodeToExpr(branches.head) + CaseKeyWhen(keyExpr, branches.drop(1).map(nodeToExpr)) /* Complex datatype manipulation */ case Token("[", child :: ordinal :: Nil) => diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index 616352d223c5b..c605f1017547a 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -751,4 +751,11 @@ class SQLQuerySuite extends QueryTest { (6, "c", 0, 6) ).map(i => Row(i._1, i._2, i._3, i._4))) } + + test("test case key when") { + (1 to 5).map(i => (i, i.toString)).toDF("k", "v").registerTempTable("t") + checkAnswer( + sql("SELECT CASE k WHEN 2 THEN 22 WHEN 4 THEN 44 ELSE 0 END, v FROM t"), + Row(0, "1") :: Row(22, "2") :: Row(0, "3") :: Row(44, "4") :: Row(0, "5") :: Nil) + } } From 88063c62689135da56ee6678b2e826b88c382732 Mon Sep 17 00:00:00 2001 From: tedyu Date: Thu, 7 May 2015 16:53:59 -0700 Subject: [PATCH 50/75] [SPARK-7450] Use UNSAFE.getLong() to speed up BitSetMethods#anySet() Author: tedyu Closes #5897 from tedyu/master and squashes the following commits: 473bf9d [tedyu] Address Josh's review comments 1719c5b [tedyu] Correct upper bound in for loop b51dcaf [tedyu] Add unit test in BitSetSuite for BitSet#anySet() 83f9f87 [tedyu] Merge branch 'master' of github.com:apache/spark 817e3f9 [tedyu] Replace constant 8 with SIZE_OF_LONG 75a467b [tedyu] Correct offset for UNSAFE.getLong() 855374b [tedyu] Remove second loop since bitSetWidthInBytes is WORD aligned 093b7a4 [tedyu] Use UNSAFE.getLong() to speed up BitSetMethods#anySet() 63ee050 [tedyu] Use UNSAFE.getLong() to speed up BitSetMethods#anySet() 4ca0ef6 [tedyu] Use UNSAFE.getLong() to speed up BitSetMethods#anySet() 3e9b6919 [tedyu] Use UNSAFE.getLong() to speed up BitSetMethods#anySet() --- .../main/java/org/apache/spark/unsafe/bitset/BitSet.java | 8 ++++++++ .../org/apache/spark/unsafe/bitset/BitSetMethods.java | 7 ++++--- .../java/org/apache/spark/unsafe/bitset/BitSetSuite.java | 7 +++++++ 3 files changed, 19 insertions(+), 3 deletions(-) diff --git a/unsafe/src/main/java/org/apache/spark/unsafe/bitset/BitSet.java b/unsafe/src/main/java/org/apache/spark/unsafe/bitset/BitSet.java index f72e07fce92fd..28e23da108ebe 100644 --- a/unsafe/src/main/java/org/apache/spark/unsafe/bitset/BitSet.java +++ b/unsafe/src/main/java/org/apache/spark/unsafe/bitset/BitSet.java @@ -102,4 +102,12 @@ public boolean isSet(int index) { public int nextSetBit(int fromIndex) { return BitSetMethods.nextSetBit(baseObject, baseOffset, fromIndex, numWords); } + + /** + * Returns {@code true} if any bit is set. + */ + public boolean anySet() { + return BitSetMethods.anySet(baseObject, baseOffset, numWords); + } + } diff --git a/unsafe/src/main/java/org/apache/spark/unsafe/bitset/BitSetMethods.java b/unsafe/src/main/java/org/apache/spark/unsafe/bitset/BitSetMethods.java index f30626d8f4317..0987191c1c636 100644 --- a/unsafe/src/main/java/org/apache/spark/unsafe/bitset/BitSetMethods.java +++ b/unsafe/src/main/java/org/apache/spark/unsafe/bitset/BitSetMethods.java @@ -70,9 +70,10 @@ public static boolean isSet(Object baseObject, long baseOffset, int index) { /** * Returns {@code true} if any bit is set. */ - public static boolean anySet(Object baseObject, long baseOffset, long bitSetWidthInBytes) { - for (int i = 0; i <= bitSetWidthInBytes; i++) { - if (PlatformDependent.UNSAFE.getByte(baseObject, baseOffset + i) != 0) { + public static boolean anySet(Object baseObject, long baseOffset, long bitSetWidthInWords) { + long addr = baseOffset; + for (int i = 0; i < bitSetWidthInWords; i++, addr += WORD_SIZE) { + if (PlatformDependent.UNSAFE.getLong(baseObject, addr) != 0) { return true; } } diff --git a/unsafe/src/test/java/org/apache/spark/unsafe/bitset/BitSetSuite.java b/unsafe/src/test/java/org/apache/spark/unsafe/bitset/BitSetSuite.java index e3a824e29b768..18393db9f382f 100644 --- a/unsafe/src/test/java/org/apache/spark/unsafe/bitset/BitSetSuite.java +++ b/unsafe/src/test/java/org/apache/spark/unsafe/bitset/BitSetSuite.java @@ -39,6 +39,8 @@ public void basicOps() { for (int i = 0; i < bs.capacity(); i++) { Assert.assertFalse(bs.isSet(i)); } + // another form of asserting that the bit set is empty + Assert.assertFalse(bs.anySet()); // Set every bit and check it. for (int i = 0; i < bs.capacity(); i++) { @@ -52,6 +54,11 @@ public void basicOps() { bs.unset(i); Assert.assertFalse(bs.isSet(i)); } + + // Make sure anySet() can detect any set bit + bs = createBitSet(256); + bs.set(64); + Assert.assertTrue(bs.anySet()); } @Test From 22ab70e06ede65ca865073fe36c859042a920aa3 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Thu, 7 May 2015 17:34:44 -0700 Subject: [PATCH 51/75] [SPARK-7305] [STREAMING] [WEBUI] Make BatchPage show friendly information when jobs are dropped by SparkListener If jobs are dropped by SparkListener, at least we can show the job ids in BatchPage. Screenshot: ![b1](https://cloud.githubusercontent.com/assets/1000778/7434968/f19aa784-eff3-11e4-8f86-36a073873574.png) Author: zsxwing Closes #5840 from zsxwing/SPARK-7305 and squashes the following commits: aca0ba6 [zsxwing] Fix the code style 718765e [zsxwing] Make generateNormalJobRow private 8073b03 [zsxwing] Merge branch 'master' into SPARK-7305 83dec11 [zsxwing] Make BatchPage show friendly information when jobs are dropped by SparkListener --- .../apache/spark/streaming/ui/BatchPage.scala | 136 ++++++++++++++---- 1 file changed, 106 insertions(+), 30 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/BatchPage.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/BatchPage.scala index 3f1cab69068dc..831f60e870f74 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/BatchPage.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/BatchPage.scala @@ -19,7 +19,7 @@ package org.apache.spark.streaming.ui import javax.servlet.http.HttpServletRequest -import scala.xml.{NodeSeq, Node} +import scala.xml.{NodeSeq, Node, Text} import org.apache.commons.lang3.StringEscapeUtils @@ -28,6 +28,7 @@ import org.apache.spark.ui.{UIUtils => SparkUIUtils, WebUIPage} import org.apache.spark.streaming.ui.StreamingJobProgressListener.{SparkJobId, OutputOpId} import org.apache.spark.ui.jobs.UIData.JobUIData +private case class SparkJobIdWithUIData(sparkJobId: SparkJobId, jobUIData: Option[JobUIData]) private[ui] class BatchPage(parent: StreamingTab) extends WebUIPage("batch") { private val streamingListener = parent.listener @@ -44,25 +45,33 @@ private[ui] class BatchPage(parent: StreamingTab) extends WebUIPage("batch") { Error } + private def generateJobRow( + outputOpId: OutputOpId, + outputOpDescription: Seq[Node], + formattedOutputOpDuration: String, + numSparkJobRowsInOutputOp: Int, + isFirstRow: Boolean, + sparkJob: SparkJobIdWithUIData): Seq[Node] = { + if (sparkJob.jobUIData.isDefined) { + generateNormalJobRow(outputOpId, outputOpDescription, formattedOutputOpDuration, + numSparkJobRowsInOutputOp, isFirstRow, sparkJob.jobUIData.get) + } else { + generateDroppedJobRow(outputOpId, outputOpDescription, formattedOutputOpDuration, + numSparkJobRowsInOutputOp, isFirstRow, sparkJob.sparkJobId) + } + } + /** * Generate a row for a Spark Job. Because duplicated output op infos needs to be collapsed into * one cell, we use "rowspan" for the first row of a output op. */ - def generateJobRow( + private def generateNormalJobRow( outputOpId: OutputOpId, + outputOpDescription: Seq[Node], formattedOutputOpDuration: String, numSparkJobRowsInOutputOp: Int, isFirstRow: Boolean, sparkJob: JobUIData): Seq[Node] = { - val lastStageInfo = Option(sparkJob.stageIds) - .filter(_.nonEmpty) - .flatMap { ids => sparkListener.stageIdToInfo.get(ids.max) } - val lastStageData = lastStageInfo.flatMap { s => - sparkListener.stageIdToData.get((s.stageId, s.attemptId)) - } - - val lastStageName = lastStageInfo.map(_.name).getOrElse("(Unknown Stage Name)") - val lastStageDescription = lastStageData.flatMap(_.description).getOrElse("") val duration: Option[Long] = { sparkJob.submissionTime.map { start => val end = sparkJob.completionTime.getOrElse(System.currentTimeMillis()) @@ -83,9 +92,7 @@ private[ui] class BatchPage(parent: StreamingTab) extends WebUIPage("batch") { if (isFirstRow) { {outputOpId.toString} - - {lastStageDescription} - {lastStageName} + {outputOpDescription} {formattedOutputOpDuration} } else { @@ -122,27 +129,96 @@ private[ui] class BatchPage(parent: StreamingTab) extends WebUIPage("batch") { } - private def generateOutputOpIdRow( - outputOpId: OutputOpId, sparkJobs: Seq[JobUIData]): Seq[Node] = { - val sparkjobDurations = sparkJobs.map(sparkJob => { - sparkJob.submissionTime.map { start => - val end = sparkJob.completionTime.getOrElse(System.currentTimeMillis()) - end - start + /** + * If a job is dropped by sparkListener due to exceeding the limitation, we only show the job id + * with "-" cells. + */ + private def generateDroppedJobRow( + outputOpId: OutputOpId, + outputOpDescription: Seq[Node], + formattedOutputOpDuration: String, + numSparkJobRowsInOutputOp: Int, + isFirstRow: Boolean, + jobId: Int): Seq[Node] = { + // In the first row, output op id and its information needs to be shown. In other rows, these + // cells will be taken up due to "rowspan". + // scalastyle:off + val prefixCells = + if (isFirstRow) { + {outputOpId.toString} + {outputOpDescription} + {formattedOutputOpDuration} + } else { + Nil } - }) + // scalastyle:on + + + {prefixCells} + + {jobId.toString} + + + - + + - + + - + + - + + } + + private def generateOutputOpIdRow( + outputOpId: OutputOpId, sparkJobs: Seq[SparkJobIdWithUIData]): Seq[Node] = { + // We don't count the durations of dropped jobs + val sparkJobDurations = sparkJobs.filter(_.jobUIData.nonEmpty).map(_.jobUIData.get). + map(sparkJob => { + sparkJob.submissionTime.map { start => + val end = sparkJob.completionTime.getOrElse(System.currentTimeMillis()) + end - start + } + }) val formattedOutputOpDuration = - if (sparkjobDurations.exists(_ == None)) { - // If any job does not finish, set "formattedOutputOpDuration" to "-" + if (sparkJobDurations.isEmpty || sparkJobDurations.exists(_ == None)) { + // If no job or any job does not finish, set "formattedOutputOpDuration" to "-" "-" } else { - SparkUIUtils.formatDuration(sparkjobDurations.flatMap(x => x).sum) + SparkUIUtils.formatDuration(sparkJobDurations.flatMap(x => x).sum) } - generateJobRow(outputOpId, formattedOutputOpDuration, sparkJobs.size, true, sparkJobs.head) ++ + + val description = generateOutputOpDescription(sparkJobs) + + generateJobRow( + outputOpId, description, formattedOutputOpDuration, sparkJobs.size, true, sparkJobs.head) ++ sparkJobs.tail.map { sparkJob => - generateJobRow(outputOpId, formattedOutputOpDuration, sparkJobs.size, false, sparkJob) + generateJobRow( + outputOpId, description, formattedOutputOpDuration, sparkJobs.size, false, sparkJob) }.flatMap(x => x) } + private def generateOutputOpDescription(sparkJobs: Seq[SparkJobIdWithUIData]): Seq[Node] = { + val lastStageInfo = + sparkJobs.flatMap(_.jobUIData).headOption. // Get the first JobUIData + flatMap { sparkJob => // For the first job, get the latest Stage info + if (sparkJob.stageIds.isEmpty) { + None + } else { + sparkListener.stageIdToInfo.get(sparkJob.stageIds.max) + } + } + val lastStageData = lastStageInfo.flatMap { s => + sparkListener.stageIdToData.get((s.stageId, s.attemptId)) + } + + val lastStageName = lastStageInfo.map(_.name).getOrElse("(Unknown Stage Name)") + val lastStageDescription = lastStageData.flatMap(_.description).getOrElse("") + + + {lastStageDescription} + ++ Text(lastStageName) + } + private def failureReasonCell(failureReason: String): Seq[Node] = { val isMultiline = failureReason.indexOf('\n') >= 0 // Display the first line by default @@ -187,10 +263,10 @@ private[ui] class BatchPage(parent: StreamingTab) extends WebUIPage("batch") { (outputOpId, outputOpIdAndSparkJobIds.map(_.sparkJobId).sorted) } sparkListener.synchronized { - val outputOpIdWithJobs: Seq[(OutputOpId, Seq[JobUIData])] = + val outputOpIdWithJobs: Seq[(OutputOpId, Seq[SparkJobIdWithUIData])] = outputOpIdToSparkJobIds.map { case (outputOpId, sparkJobIds) => - // Filter out spark Job ids that don't exist in sparkListener - (outputOpId, sparkJobIds.flatMap(getJobData)) + (outputOpId, + sparkJobIds.map(sparkJobId => SparkJobIdWithUIData(sparkJobId, getJobData(sparkJobId)))) } @@ -200,7 +276,7 @@ private[ui] class BatchPage(parent: StreamingTab) extends WebUIPage("batch") { { outputOpIdWithJobs.map { - case (outputOpId, jobs) => generateOutputOpIdRow(outputOpId, jobs) + case (outputOpId, sparkJobIds) => generateOutputOpIdRow(outputOpId, sparkJobIds) } } From cd1d4110cfffb413ab585cf1cc8f1264243cb393 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 7 May 2015 19:36:24 -0700 Subject: [PATCH 52/75] [SPARK-6908] [SQL] Use isolated Hive client This PR switches Spark SQL's Hive support to use the isolated hive client interface introduced by #5851, instead of directly interacting with the client. By using this isolated client we can now allow users to dynamically configure the version of Hive that they are connecting to by setting `spark.sql.hive.metastore.version` without the need recompile. This also greatly reduces the surface area for our interaction with the hive libraries, hopefully making it easier to support other versions in the future. Jars for the desired hive version can be configured using `spark.sql.hive.metastore.jars`, which accepts the following options: - a colon-separated list of jar files or directories for hive and hadoop. - `builtin` - attempt to discover the jars that were used to load Spark SQL and use those. This option is only valid when using the execution version of Hive. - `maven` - download the correct version of hive on demand from maven. By default, `builtin` is used for Hive 13. This PR also removes the test step for building against Hive 12, as this will no longer be required to talk to Hive 12 metastores. However, the full removal of the Shim is deferred until a later PR. Remaining TODOs: - Remove the Hive Shims and inline code for Hive 13. - Several HiveCompatibility tests are not yet passing. - `nullformatCTAS` - As detailed below, we now are handling CTAS parsing ourselves instead of hacking into the Hive semantic analyzer. However, we currently only handle the common cases and not things like CTAS where the null format is specified. - `combine1` now leaks state about compression somehow, breaking all subsequent tests. As such we currently add it to the blacklist - `part_inherit_tbl_props` and `part_inherit_tbl_props_with_star` do not work anymore. We are correctly propagating the information - "load_dyn_part14.*" - These tests pass when run on their own, but fail when run with all other tests. It seems our `RESET` mechanism may not be as robust as it used to be? Other required changes: - `CreateTableAsSelect` no longer carries parts of the HiveQL AST with it through the query execution pipeline. Instead, we parse CTAS during the HiveQL conversion and construct a `HiveTable`. The full parsing here is not yet complete as detailed above in the remaining TODOs. Since the operator is Hive specific, it is moved to the hive package. - `Command` is simplified to be a trait that simply acts as a marker for a LogicalPlan that should be eagerly evaluated. Author: Michael Armbrust Closes #5876 from marmbrus/useIsolatedClient and squashes the following commits: 258d000 [Michael Armbrust] really really correct path handling e56fd4a [Michael Armbrust] getAbsolutePath 5a259f5 [Michael Armbrust] fix typos 81bb366 [Michael Armbrust] comments from vanzin 5f3945e [Michael Armbrust] Merge remote-tracking branch 'origin/master' into useIsolatedClient 4b5cd41 [Michael Armbrust] yin's comments f5de7de [Michael Armbrust] cleanup 11e9c72 [Michael Armbrust] better coverage in versions suite 7e8f010 [Michael Armbrust] better error messages and jar handling e7b3941 [Michael Armbrust] more permisive checking for function registration da91ba7 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into useIsolatedClient 5fe5894 [Michael Armbrust] fix serialization suite 81711c4 [Michael Armbrust] Initial support for running without maven 1d8ae44 [Michael Armbrust] fix final tests? 1c50813 [Michael Armbrust] more comments a3bee70 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into useIsolatedClient a6f5df1 [Michael Armbrust] style ab07f7e [Michael Armbrust] WIP 4d8bf02 [Michael Armbrust] Remove hive 12 compilation 8843a25 [Michael Armbrust] [SPARK-6908] [SQL] Use isolated Hive client --- dev/run-tests | 23 - project/MimaExcludes.scala | 2 + project/SparkBuild.scala | 9 +- .../plans/logical/basicOperators.scala | 16 +- .../sql/catalyst/plans/logical/commands.scala | 8 +- .../spark/sql/catalyst/SqlParserSuite.scala | 6 +- .../org/apache/spark/sql/DataFrame.scala | 1 - .../org/apache/spark/sql/SQLContext.scala | 11 +- .../apache/spark/sql/execution/commands.scala | 4 +- .../org/apache/spark/sql/sources/ddl.scala | 16 +- .../hive/thriftserver/SparkSQLCLIDriver.scala | 26 +- .../sql/hive/thriftserver/SparkSQLEnv.scala | 9 +- .../execution/HiveCompatibilitySuite.scala | 12 +- .../apache/spark/sql/hive/HiveContext.scala | 283 ++++++------ .../spark/sql/hive/HiveMetastoreCatalog.scala | 415 ++++++------------ .../org/apache/spark/sql/hive/HiveQl.scala | 126 +++++- .../apache/spark/sql/hive/TableReader.scala | 11 +- .../sql/hive/client/ClientInterface.scala | 41 +- .../spark/sql/hive/client/ClientWrapper.scala | 99 +++-- .../hive/client/IsolatedClientLoader.scala | 23 +- .../sql/hive/client/ReflectionMagic.scala | 8 + .../hive/execution/CreateTableAsSelect.scala | 33 +- .../hive/execution/InsertIntoHiveTable.scala | 33 +- .../spark/sql/hive/execution/commands.scala | 13 + .../apache/spark/sql/hive/test/TestHive.scala | 72 ++- sql/hive/src/test/resources/log4j.properties | 2 +- .../spark/sql/hive/ErrorPositionSuite.scala | 22 +- .../sql/hive/MetastoreDataSourcesSuite.scala | 26 +- .../spark/sql/hive/SerializationSuite.scala | 6 +- .../spark/sql/hive/client/VersionsSuite.scala | 78 +++- .../hive/execution/HiveComparisonTest.scala | 2 + .../sql/hive/execution/HiveQuerySuite.scala | 2 +- .../sql/hive/execution/PruningSuite.scala | 15 +- 33 files changed, 782 insertions(+), 671 deletions(-) diff --git a/dev/run-tests b/dev/run-tests index 05c63bce4d40d..ef587a1a5988c 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -142,29 +142,6 @@ CURRENT_BLOCK=$BLOCK_BUILD { HIVE_BUILD_ARGS="$SBT_MAVEN_PROFILES_ARGS -Phive -Phive-thriftserver" - HIVE_12_BUILD_ARGS="$HIVE_BUILD_ARGS -Phive-0.12.0" - - # First build with Hive 0.12.0 to ensure patches do not break the Hive 0.12.0 build - echo "[info] Compile with Hive 0.12.0" - [ -d "lib_managed" ] && rm -rf lib_managed - echo "[info] Building Spark with these arguments: $HIVE_12_BUILD_ARGS" - - if [ "${AMPLAB_JENKINS_BUILD_TOOL}" == "maven" ]; then - build/mvn $HIVE_12_BUILD_ARGS clean package -DskipTests - else - # NOTE: echo "q" is needed because sbt on encountering a build file with failure - # (either resolution or compilation) prompts the user for input either q, r, etc - # to quit or retry. This echo is there to make it not block. - # NOTE: Do not quote $BUILD_MVN_PROFILE_ARGS or else it will be interpreted as a - # single argument! - # QUESTION: Why doesn't 'yes "q"' work? - # QUESTION: Why doesn't 'grep -v -e "^\[info\] Resolving"' work? - echo -e "q\n" \ - | build/sbt $HIVE_12_BUILD_ARGS clean hive/compile hive-thriftserver/compile \ - | grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including" - fi - - # Then build with default Hive version (0.13.1) because tests are based on this version echo "[info] Compile with Hive 0.13.1" [ -d "lib_managed" ] && rm -rf lib_managed echo "[info] Building Spark with these arguments: $HIVE_BUILD_ARGS" diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index bf343d4b7e40b..cfe387faec14b 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -89,6 +89,8 @@ object MimaExcludes { ProblemFilters.exclude[MissingMethodProblem]( "org.apache.spark.mllib.linalg.Vector.numActives") ) ++ Seq( + // Execution should never be included as its always internal. + MimaBuild.excludeSparkPackage("sql.execution"), // This `protected[sql]` method was removed in 1.3.1 ProblemFilters.exclude[MissingMethodProblem]( "org.apache.spark.sql.SQLContext.checkAnalysis"), diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index b4431c7ee05b6..026855f8f6a5a 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -193,6 +193,7 @@ object SparkBuild extends PomBuild { * Usage: `build/sbt sparkShell` */ val sparkShell = taskKey[Unit]("start a spark-shell.") + val sparkSql = taskKey[Unit]("starts the spark sql CLI.") enable(Seq( connectInput in run := true, @@ -203,6 +204,12 @@ object SparkBuild extends PomBuild { sparkShell := { (runMain in Compile).toTask(" org.apache.spark.repl.Main -usejavacp").value + }, + + javaOptions in Compile += "-Dspark.master=local", + + sparkSql := { + (runMain in Compile).toTask(" org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver").value } ))(assembly) @@ -497,7 +504,7 @@ object TestSettings { // Setting SPARK_DIST_CLASSPATH is a simple way to make sure any child processes // launched by the tests have access to the correct test-time classpath. envVars in Test ++= Map( - "SPARK_DIST_CLASSPATH" -> + "SPARK_DIST_CLASSPATH" -> (fullClasspath in Test).value.files.map(_.getAbsolutePath).mkString(":").stripSuffix(":"), "JAVA_HOME" -> sys.env.get("JAVA_HOME").getOrElse(sys.props("java.home"))), javaOptions in Test += "-Dspark.test.home=" + sparkHome, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala index ba0abb2df596c..0f349f9d11415 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala @@ -149,16 +149,6 @@ case class InsertIntoTable( } } -case class CreateTableAsSelect[T]( - databaseName: Option[String], - tableName: String, - child: LogicalPlan, - allowExisting: Boolean, - desc: Option[T] = None) extends UnaryNode { - override def output: Seq[Attribute] = Seq.empty[Attribute] - override lazy val resolved: Boolean = databaseName != None && childrenResolved -} - /** * A container for holding named common table expressions (CTEs) and a query plan. * This operator will be removed during analysis and the relations will be substituted into child. @@ -184,10 +174,10 @@ case class WriteToFile( } /** - * @param order The ordering expressions - * @param global True means global sorting apply for entire data set, + * @param order The ordering expressions + * @param global True means global sorting apply for entire data set, * False means sorting only apply within the partition. - * @param child Child logical plan + * @param child Child logical plan */ case class Sort( order: Seq[SortOrder], diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala index 45905f8ef98c5..246f4d7e34d3d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala @@ -21,9 +21,7 @@ import org.apache.spark.sql.catalyst.expressions.Attribute /** * A logical node that represents a non-query command to be executed by the system. For example, - * commands can be used by parsers to represent DDL operations. + * commands can be used by parsers to represent DDL operations. Commands, unlike queries, are + * eagerly executed. */ -abstract class Command extends LeafNode { - self: Product => - def output: Seq[Attribute] = Seq.empty -} +trait Command diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/SqlParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/SqlParserSuite.scala index a652c70560990..890ea2a84b82e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/SqlParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/SqlParserSuite.scala @@ -17,11 +17,15 @@ package org.apache.spark.sql.catalyst +import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.plans.logical.Command import org.scalatest.FunSuite -private[sql] case class TestCommand(cmd: String) extends Command +private[sql] case class TestCommand(cmd: String) extends LogicalPlan with Command { + override def output: Seq[Attribute] = Seq.empty + override def children: Seq[LogicalPlan] = Seq.empty +} private[sql] class SuperLongKeywordTestParser extends AbstractSparkSQLParser { protected val EXECUTE = Keyword("THISISASUPERLONGKEYWORDTEST") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala index 79fbf50300d4d..7947042c14299 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala @@ -143,7 +143,6 @@ class DataFrame private[sql]( // happen right away to let these side effects take place eagerly. case _: Command | _: InsertIntoTable | - _: CreateTableAsSelect[_] | _: CreateTableUsingAsSelect | _: WriteToFile => LogicalRDD(queryExecution.analyzed.output, queryExecution.toRdd)(sqlContext) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 0563430a6fdc7..0ac0936f0f592 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -70,7 +70,7 @@ import org.apache.spark.{Partition, SparkContext} * spark-sql> SELECT * FROM src LIMIT 1; * *-- Exception will be thrown and switch to dialect - *-- "sql" (for SQLContext) or + *-- "sql" (for SQLContext) or *-- "hiveql" (for HiveContext) * }}} */ @@ -107,7 +107,7 @@ class SQLContext(@transient val sparkContext: SparkContext) /** * @return Spark SQL configuration */ - protected[sql] def conf = tlSession.get().conf + protected[sql] def conf = currentSession().conf /** * Set Spark SQL configuration properties. @@ -1197,13 +1197,17 @@ class SQLContext(@transient val sparkContext: SparkContext) |${stringOrError(executedPlan)} """.stripMargin.trim - override def toString: String = + override def toString: String = { + def output = + analyzed.output.map(o => s"${o.name}: ${o.dataType.simpleString}").mkString(", ") + // TODO previously will output RDD details by run (${stringOrError(toRdd.toDebugString)}) // however, the `toRdd` will cause the real execution, which is not what we want. // We need to think about how to avoid the side effect. s"""== Parsed Logical Plan == |${stringOrError(logical)} |== Analyzed Logical Plan == + |${stringOrError(output)} |${stringOrError(analyzed)} |== Optimized Logical Plan == |${stringOrError(optimizedPlan)} @@ -1212,6 +1216,7 @@ class SQLContext(@transient val sparkContext: SparkContext) |Code Generation: ${stringOrError(executedPlan.codegenEnabled)} |== RDD == """.stripMargin.trim + } } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala index 65687db4e6235..388a8184e4cfe 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala @@ -32,9 +32,11 @@ import org.apache.spark.sql.{DataFrame, SQLConf, SQLContext} * A logical command that is executed for its side-effects. `RunnableCommand`s are * wrapped in `ExecutedCommand` during execution. */ -trait RunnableCommand extends logical.Command { +private[sql] trait RunnableCommand extends LogicalPlan with logical.Command { self: Product => + override def output: Seq[Attribute] = Seq.empty + override def children: Seq[LogicalPlan] = Seq.empty def run(sqlContext: SQLContext): Seq[Row] } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala index 1abf3aa51cb25..06c64f2bdd59e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala @@ -269,8 +269,10 @@ private[sql] case class ResolvedDataSource(provider: Class[_], relation: BaseRel */ private[sql] case class DescribeCommand( table: LogicalPlan, - isExtended: Boolean) extends Command { - override val output = Seq( + isExtended: Boolean) extends LogicalPlan with Command { + + override def children: Seq[LogicalPlan] = Seq.empty + override val output: Seq[Attribute] = Seq( // Column names are based on Hive. AttributeReference("col_name", StringType, nullable = false, new MetadataBuilder().putString("comment", "name of the column").build())(), @@ -292,7 +294,11 @@ private[sql] case class CreateTableUsing( temporary: Boolean, options: Map[String, String], allowExisting: Boolean, - managedIfNoPath: Boolean) extends Command + managedIfNoPath: Boolean) extends LogicalPlan with Command { + + override def output: Seq[Attribute] = Seq.empty + override def children: Seq[LogicalPlan] = Seq.empty +} /** * A node used to support CTAS statements and saveAsTable for the data source API. @@ -318,7 +324,7 @@ private[sql] case class CreateTempTableUsing( provider: String, options: Map[String, String]) extends RunnableCommand { - def run(sqlContext: SQLContext): Seq[Row] = { + override def run(sqlContext: SQLContext): Seq[Row] = { val resolved = ResolvedDataSource(sqlContext, userSpecifiedSchema, provider, options) sqlContext.registerDataFrameAsTable( DataFrame(sqlContext, LogicalRelation(resolved.relation)), tableName) @@ -333,7 +339,7 @@ private[sql] case class CreateTempTableUsingAsSelect( options: Map[String, String], query: LogicalPlan) extends RunnableCommand { - def run(sqlContext: SQLContext): Seq[Row] = { + override def run(sqlContext: SQLContext): Seq[Row] = { val df = DataFrame(sqlContext, query) val resolved = ResolvedDataSource(sqlContext, provider, mode, options, df) sqlContext.registerDataFrameAsTable( diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala index b7b6925aa87f7..deb1008c468bf 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala @@ -37,7 +37,7 @@ import org.apache.hadoop.hive.ql.session.SessionState import org.apache.thrift.transport.TSocket import org.apache.spark.Logging -import org.apache.spark.sql.hive.HiveShim +import org.apache.spark.sql.hive.{HiveContext, HiveShim} import org.apache.spark.util.Utils private[hive] object SparkSQLCLIDriver { @@ -74,7 +74,12 @@ private[hive] object SparkSQLCLIDriver { System.exit(1) } - val sessionState = new CliSessionState(new HiveConf(classOf[SessionState])) + val cliConf = new HiveConf(classOf[SessionState]) + // Override the location of the metastore since this is only used for local execution. + HiveContext.newTemporaryConfiguration().foreach { + case (key, value) => cliConf.set(key, value) + } + val sessionState = new CliSessionState(cliConf) sessionState.in = System.in try { @@ -91,10 +96,14 @@ private[hive] object SparkSQLCLIDriver { // Set all properties specified via command line. val conf: HiveConf = sessionState.getConf - sessionState.cmdProperties.entrySet().foreach { item: java.util.Map.Entry[Object, Object] => - conf.set(item.getKey.asInstanceOf[String], item.getValue.asInstanceOf[String]) - sessionState.getOverriddenConfigurations.put( - item.getKey.asInstanceOf[String], item.getValue.asInstanceOf[String]) + sessionState.cmdProperties.entrySet().foreach { item => + val key = item.getKey.asInstanceOf[String] + val value = item.getValue.asInstanceOf[String] + // We do not propagate metastore options to the execution copy of hive. + if (key != "javax.jdo.option.ConnectionURL") { + conf.set(key, value) + sessionState.getOverriddenConfigurations.put(key, value) + } } SessionState.start(sessionState) @@ -138,8 +147,9 @@ private[hive] object SparkSQLCLIDriver { case e: UnsupportedEncodingException => System.exit(3) } - // use the specified database if specified - cli.processSelectDatabase(sessionState); + if (sessionState.database != null) { + SparkSQLEnv.hiveContext.runSqlHive(s"USE ${sessionState.database}") + } // Execute -i init files (always in silent mode) cli.processInitFiles(sessionState) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala index 97b46a01ba5b4..7c0c505e2d61e 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.hive.thriftserver +import java.io.PrintStream + import scala.collection.JavaConversions._ import org.apache.spark.scheduler.StatsReportListener @@ -39,7 +41,6 @@ private[hive] object SparkSQLEnv extends Logging { sparkConf .setAppName(s"SparkSQL::${Utils.localHostName()}") - .set("spark.sql.hive.version", HiveShim.version) .set( "spark.serializer", maybeSerializer.getOrElse("org.apache.spark.serializer.KryoSerializer")) @@ -51,6 +52,12 @@ private[hive] object SparkSQLEnv extends Logging { sparkContext.addSparkListener(new StatsReportListener()) hiveContext = new HiveContext(sparkContext) + hiveContext.metadataHive.setOut(new PrintStream(System.out, true, "UTF-8")) + hiveContext.metadataHive.setInfo(new PrintStream(System.err, true, "UTF-8")) + hiveContext.metadataHive.setError(new PrintStream(System.err, true, "UTF-8")) + + hiveContext.setConf("spark.sql.hive.version", HiveShim.version) + if (log.isDebugEnabled) { hiveContext.hiveconf.getAllProperties.toSeq.sorted.foreach { case (k, v) => logDebug(s"HiveConf var: $k=$v") diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index 5e411c2fdba9d..b6245a57074c8 100644 --- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -240,7 +240,17 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { // It has a bug and it has been fixed by // https://issues.apache.org/jira/browse/HIVE-7673 (in Hive 0.14 and trunk). - "input46" + "input46", + + // These tests were broken by the hive client isolation PR. + "part_inherit_tbl_props", + "part_inherit_tbl_props_with_star", + + "nullformatCTAS", // SPARK-7411: need to finish CTAS parser + + // The isolated classloader seemed to make some of our test reset mechanisms less robust. + "combine1", // This test changes compression settings in a way that breaks all subsequent tests. + "load_dyn_part14.*" // These work alone but fail when run with other tests... ) ++ HiveShim.compatibilityBlackList /** diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index f25723e53f07c..538c6c7f0a200 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -17,8 +17,9 @@ package org.apache.spark.sql.hive -import java.io.{BufferedReader, InputStreamReader, PrintStream} +import java.io.{BufferedReader, File, InputStreamReader, PrintStream} import java.sql.Timestamp +import java.util.{ArrayList => JArrayList} import org.apache.hadoop.hive.ql.parse.VariableSubstitution import org.apache.spark.sql.catalyst.Dialect @@ -35,15 +36,19 @@ import org.apache.hadoop.hive.ql.processors._ import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hadoop.hive.serde2.io.{DateWritable, TimestampWritable} -import org.apache.spark.SparkContext +import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.annotation.Experimental +import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.analysis.{Analyzer, EliminateSubQueries, OverrideCatalog, OverrideFunctionRegistry} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.execution.{ExecutedCommand, ExtractPythonUdfs, QueryExecutionException, SetCommand} +import org.apache.spark.sql.hive.client._ import org.apache.spark.sql.hive.execution.{DescribeHiveTableCommand, HiveNativeCommand} import org.apache.spark.sql.sources.{DDLParser, DataSourceStrategy} import org.apache.spark.sql.types._ +import org.apache.spark.util.Utils + /** * This is the HiveQL Dialect, this dialect is strongly bind with HiveContext @@ -61,6 +66,8 @@ private[hive] class HiveQLDialect extends Dialect { class HiveContext(sc: SparkContext) extends SQLContext(sc) { self => + import HiveContext._ + /** * When true, enables an experimental feature where metastore tables that use the parquet SerDe * are automatically converted to use the Spark SQL parquet table scan, instead of the Hive @@ -93,9 +100,118 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { protected[sql] def convertCTAS: Boolean = getConf("spark.sql.hive.convertCTAS", "false").toBoolean + /** + * The version of the hive client that will be used to communicate with the metastore. Note that + * this does not necessarily need to be the same version of Hive that is used internally by + * Spark SQL for execution. + */ + protected[hive] def hiveMetastoreVersion: String = + getConf(HIVE_METASTORE_VERSION, hiveExecutionVersion) + + /** + * The location of the jars that should be used to instantiate the HiveMetastoreClient. This + * property can be one of three options: + * - a classpath in the standard format for both hive and hadoop. + * - builtin - attempt to discover the jars that were used to load Spark SQL and use those. This + * option is only valid when using the execution version of Hive. + * - maven - download the correct version of hive on demand from maven. + */ + protected[hive] def hiveMetastoreJars: String = + getConf(HIVE_METASTORE_JARS, "builtin") + @transient protected[sql] lazy val substitutor = new VariableSubstitution() + /** + * The copy of the hive client that is used for execution. Currently this must always be + * Hive 13 as this is the version of Hive that is packaged with Spark SQL. This copy of the + * client is used for execution related tasks like registering temporary functions or ensuring + * that the ThreadLocal SessionState is correctly populated. This copy of Hive is *not* used + * for storing peristent metadata, and only point to a dummy metastore in a temporary directory. + */ + @transient + protected[hive] lazy val executionHive: ClientWrapper = { + logInfo(s"Initilizing execution hive, version $hiveExecutionVersion") + new ClientWrapper( + version = IsolatedClientLoader.hiveVersion(hiveExecutionVersion), + config = newTemporaryConfiguration()) + } + SessionState.setCurrentSessionState(executionHive.state) + + /** + * The copy of the Hive client that is used to retrieve metadata from the Hive MetaStore. + * The version of the Hive client that is used here must match the metastore that is configured + * in the hive-site.xml file. + */ + @transient + protected[hive] lazy val metadataHive: ClientInterface = { + val metaVersion = IsolatedClientLoader.hiveVersion(hiveMetastoreVersion) + + // We instantiate a HiveConf here to read in the hive-site.xml file and then pass the options + // into the isolated client loader + val metadataConf = new HiveConf() + // `configure` goes second to override other settings. + val allConfig = metadataConf.iterator.map(e => e.getKey -> e.getValue).toMap ++ configure + + val isolatedLoader = if (hiveMetastoreJars == "builtin") { + if (hiveExecutionVersion != hiveMetastoreVersion) { + throw new IllegalArgumentException( + "Builtin jars can only be used when hive execution version == hive metastore version. " + + s"Execution: ${hiveExecutionVersion} != Metastore: ${hiveMetastoreVersion}. " + + "Specify a vaild path to the correct hive jars using $HIVE_METASTORE_JARS " + + s"or change $HIVE_METASTORE_VERSION to $hiveExecutionVersion.") + } + val jars = getClass.getClassLoader match { + case urlClassLoader: java.net.URLClassLoader => urlClassLoader.getURLs + case other => + throw new IllegalArgumentException( + "Unable to locate hive jars to connect to metastore " + + s"using classloader ${other.getClass.getName}. " + + "Please set spark.sql.hive.metastore.jars") + } + + logInfo( + s"Initializing HiveMetastoreConnection version $hiveMetastoreVersion using Spark classes.") + new IsolatedClientLoader( + version = metaVersion, + execJars = jars.toSeq, + config = allConfig, + isolationOn = true) + } else if (hiveMetastoreJars == "maven") { + // TODO: Support for loading the jars from an already downloaded location. + logInfo( + s"Initializing HiveMetastoreConnection version $hiveMetastoreVersion using maven.") + IsolatedClientLoader.forVersion(hiveMetastoreVersion, allConfig ) + } else { + // Convert to files and expand any directories. + val jars = + hiveMetastoreJars + .split(File.pathSeparator) + .flatMap { + case path if new File(path).getName() == "*" => + val files = new File(path).getParentFile().listFiles() + if (files == null) { + logWarning(s"Hive jar path '$path' does not exist.") + Nil + } else { + files.filter(_.getName().toLowerCase().endsWith(".jar")) + } + case path => + new File(path) :: Nil + } + .map(_.toURI.toURL) + + logInfo( + s"Initializing HiveMetastoreConnection version $hiveMetastoreVersion using $jars") + new IsolatedClientLoader( + version = metaVersion, + execJars = jars.toSeq, + config = allConfig, + isolationOn = true) + } + isolatedLoader.client + } + protected[sql] override def parseSql(sql: String): LogicalPlan = { super.parseSql(substitutor.substitute(hiveconf, sql)) } @@ -178,15 +294,10 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { // recorded in the Hive metastore. // This logic is based on org.apache.hadoop.hive.ql.exec.StatsTask.aggregateStats(). if (newTotalSize > 0 && newTotalSize != oldTotalSize) { - tableParameters.put(HiveShim.getStatsSetupConstTotalSize, newTotalSize.toString) - val hiveTTable = relation.hiveQlTable.getTTable - hiveTTable.setParameters(tableParameters) - val tableFullName = - relation.hiveQlTable.getDbName + "." + relation.hiveQlTable.getTableName - - catalog.synchronized { - catalog.client.alterTable(tableFullName, new Table(hiveTTable)) - } + catalog.client.alterTable( + relation.table.copy( + properties = relation.table.properties + + (HiveShim.getStatsSetupConstTotalSize -> newTotalSize.toString))) } case otherRelation => throw new UnsupportedOperationException( @@ -194,47 +305,19 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { } } - // Circular buffer to hold what hive prints to STDOUT and ERR. Only printed when failures occur. - @transient - protected lazy val outputBuffer = new java.io.OutputStream { - var pos: Int = 0 - var buffer = new Array[Int](10240) - def write(i: Int): Unit = { - buffer(pos) = i - pos = (pos + 1) % buffer.size - } - - override def toString: String = { - val (end, start) = buffer.splitAt(pos) - val input = new java.io.InputStream { - val iterator = (start ++ end).iterator - - def read(): Int = if (iterator.hasNext) iterator.next() else -1 - } - val reader = new BufferedReader(new InputStreamReader(input)) - val stringBuilder = new StringBuilder - var line = reader.readLine() - while(line != null) { - stringBuilder.append(line) - stringBuilder.append("\n") - line = reader.readLine() - } - stringBuilder.toString() - } - } - - protected[hive] def sessionState = tlSession.get().asInstanceOf[this.SQLSession].sessionState - protected[hive] def hiveconf = tlSession.get().asInstanceOf[this.SQLSession].hiveconf override def setConf(key: String, value: String): Unit = { super.setConf(key, value) - runSqlHive(s"SET $key=$value") + hiveconf.set(key, value) + executionHive.runSqlHive(s"SET $key=$value") + metadataHive.runSqlHive(s"SET $key=$value") } /* A catalyst metadata catalog that points to the Hive Metastore. */ @transient - override protected[sql] lazy val catalog = new HiveMetastoreCatalog(this) with OverrideCatalog + override protected[sql] lazy val catalog = + new HiveMetastoreCatalog(metadataHive, this) with OverrideCatalog // Note that HiveUDFs will be overridden by functions registered in this context. @transient @@ -261,16 +344,14 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { new this.SQLSession() } + /** Overridden by child classes that need to set configuration before the client init. */ + protected def configure(): Map[String, String] = Map.empty + protected[hive] class SQLSession extends super.SQLSession { protected[sql] override lazy val conf: SQLConf = new SQLConf { override def dialect: String = getConf(SQLConf.DIALECT, "hiveql") } - protected[hive] lazy val hiveconf: HiveConf = { - setConf(sessionState.getConf.getAllProperties) - sessionState.getConf - } - /** * SQLConf and HiveConf contracts: * @@ -285,78 +366,12 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { state = new SessionState(new HiveConf(classOf[SessionState])) SessionState.start(state) } - if (state.out == null) { - state.out = new PrintStream(outputBuffer, true, "UTF-8") - } - if (state.err == null) { - state.err = new PrintStream(outputBuffer, true, "UTF-8") - } state } - } - - /** - * Runs the specified SQL query using Hive. - */ - protected[sql] def runSqlHive(sql: String): Seq[String] = { - val maxResults = 100000 - val results = runHive(sql, maxResults) - // It is very confusing when you only get back some of the results... - if (results.size == maxResults) sys.error("RESULTS POSSIBLY TRUNCATED") - results - } - - /** - * Execute the command using Hive and return the results as a sequence. Each element - * in the sequence is one row. - */ - protected def runHive(cmd: String, maxRows: Int = 1000): Seq[String] = synchronized { - try { - val cmd_trimmed: String = cmd.trim() - val tokens: Array[String] = cmd_trimmed.split("\\s+") - val cmd_1: String = cmd_trimmed.substring(tokens(0).length()).trim() - val proc: CommandProcessor = HiveShim.getCommandProcessor(Array(tokens(0)), hiveconf) - - // Makes sure the session represented by the `sessionState` field is activated. This implies - // Spark SQL Hive support uses a single `SessionState` for all Hive operations and breaks - // session isolation under multi-user scenarios (i.e. HiveThriftServer2). - // TODO Fix session isolation - if (SessionState.get() != sessionState) { - SessionState.start(sessionState) - } - proc match { - case driver: Driver => - val results = HiveShim.createDriverResultsArray - val response: CommandProcessorResponse = driver.run(cmd) - // Throw an exception if there is an error in query processing. - if (response.getResponseCode != 0) { - driver.close() - throw new QueryExecutionException(response.getErrorMessage) - } - driver.setMaxRows(maxRows) - driver.getResults(results) - driver.close() - HiveShim.processResults(results) - case _ => - if (sessionState.out != null) { - sessionState.out.println(tokens(0) + " " + cmd_1) - } - Seq(proc.run(cmd_1).getResponseCode.toString) - } - } catch { - case e: Exception => - logError( - s""" - |====================== - |HIVE FAILURE OUTPUT - |====================== - |${outputBuffer.toString} - |====================== - |END HIVE FAILURE OUTPUT - |====================== - """.stripMargin) - throw e + protected[hive] lazy val hiveconf: HiveConf = { + setConf(sessionState.getConf.getAllProperties) + sessionState.getConf } } @@ -391,17 +406,23 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { ) } + protected[hive] def runSqlHive(sql: String): Seq[String] = { + if (sql.toLowerCase.contains("create temporary function")) { + executionHive.runSqlHive(sql) + } else if (sql.trim.toLowerCase.startsWith("set")) { + metadataHive.runSqlHive(sql) + executionHive.runSqlHive(sql) + } else { + metadataHive.runSqlHive(sql) + } + } + @transient override protected[sql] val planner = hivePlanner /** Extends QueryExecution with hive specific features. */ protected[sql] class QueryExecution(logicalPlan: LogicalPlan) extends super.QueryExecution(logicalPlan) { - // Like what we do in runHive, makes sure the session represented by the - // `sessionState` field is activated. - if (SessionState.get() != sessionState) { - SessionState.start(sessionState) - } /** * Returns the result as a hive compatible sequence of strings. For native commands, the @@ -439,7 +460,21 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { } -private object HiveContext { +private[hive] object HiveContext { + /** The version of hive used internally by Spark SQL. */ + val hiveExecutionVersion: String = "0.13.1" + + val HIVE_METASTORE_VERSION: String = "spark.sql.hive.metastore.version" + val HIVE_METASTORE_JARS: String = "spark.sql.hive.metastore.jars" + + /** Constructs a configuration for hive, where the metastore is located in a temp directory. */ + def newTemporaryConfiguration(): Map[String, String] = { + val tempDir = Utils.createTempDir() + val localMetastore = new File(tempDir, "metastore").getAbsolutePath + Map( + "javax.jdo.option.ConnectionURL" -> s"jdbc:derby:;databaseName=$localMetastore;create=true") + } + protected val primitiveTypes = Seq(StringType, IntegerType, LongType, DoubleType, FloatType, BooleanType, ByteType, ShortType, DateType, TimestampType, BinaryType) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index 4d222cf88e5e8..8fcdf3d0ab119 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -22,6 +22,8 @@ import java.util.{List => JList} import com.google.common.base.Objects import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache} + +import org.apache.hadoop.fs.Path import org.apache.hadoop.hive.metastore.api.{FieldSchema, Partition => TPartition, Table => TTable} import org.apache.hadoop.hive.metastore.{TableType, Warehouse} import org.apache.hadoop.hive.ql.metadata._ @@ -32,6 +34,7 @@ import org.apache.hadoop.hive.serde2.{Deserializer, SerDeException} import org.apache.hadoop.util.ReflectionUtils import org.apache.spark.Logging +import org.apache.spark.sql.hive.client.IsolatedClientLoader import org.apache.spark.sql.{SaveMode, AnalysisException, SQLContext} import org.apache.spark.sql.catalyst.analysis.{MultiInstanceRelation, NoSuchTableException, Catalog, OverrideCatalog} import org.apache.spark.sql.catalyst.expressions._ @@ -39,6 +42,7 @@ import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ +import org.apache.spark.sql.hive.client._ import org.apache.spark.sql.parquet.{ParquetRelation2, Partition => ParquetPartition, PartitionSpec} import org.apache.spark.sql.sources.{CreateTableUsingAsSelect, DDLParser, LogicalRelation, ResolvedDataSource} import org.apache.spark.sql.types._ @@ -47,11 +51,10 @@ import org.apache.spark.util.Utils /* Implicit conversions */ import scala.collection.JavaConversions._ -private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with Logging { - import org.apache.spark.sql.hive.HiveMetastoreTypes._ +private[hive] class HiveMetastoreCatalog(val client: ClientInterface, hive: HiveContext) + extends Catalog with Logging { - /** Connection to hive metastore. Usages should lock on `this`. */ - protected[hive] val client = Hive.get(hive.hiveconf) + import org.apache.spark.sql.hive.HiveMetastoreTypes._ /** Usages should lock on `this`. */ protected[hive] lazy val hiveWarehouse = new Warehouse(hive.hiveconf) @@ -67,14 +70,12 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with val cacheLoader = new CacheLoader[QualifiedTableName, LogicalPlan]() { override def load(in: QualifiedTableName): LogicalPlan = { logDebug(s"Creating new cached data source for $in") - val table = HiveMetastoreCatalog.this.synchronized { - client.getTable(in.database, in.name) - } + val table = client.getTable(in.database, in.name) def schemaStringFromParts: Option[String] = { - Option(table.getProperty("spark.sql.sources.schema.numParts")).map { numParts => + table.properties.get("spark.sql.sources.schema.numParts").map { numParts => val parts = (0 until numParts.toInt).map { index => - val part = table.getProperty(s"spark.sql.sources.schema.part.${index}") + val part = table.properties.get(s"spark.sql.sources.schema.part.${index}").orNull if (part == null) { throw new AnalysisException( s"Could not read schema from the metastore because it is corrupted " + @@ -92,20 +93,20 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with // After SPARK-6024, we removed this flag. // Although we are not using spark.sql.sources.schema any more, we need to still support. val schemaString = - Option(table.getProperty("spark.sql.sources.schema")).orElse(schemaStringFromParts) + table.properties.get("spark.sql.sources.schema").orElse(schemaStringFromParts) val userSpecifiedSchema = schemaString.map(s => DataType.fromJson(s).asInstanceOf[StructType]) // It does not appear that the ql client for the metastore has a way to enumerate all the // SerDe properties directly... - val options = table.getTTable.getSd.getSerdeInfo.getParameters.toMap + val options = table.serdeProperties val resolvedRelation = ResolvedDataSource( hive, userSpecifiedSchema, - table.getProperty("spark.sql.sources.provider"), + table.properties("spark.sql.sources.provider"), options) LogicalRelation(resolvedRelation.relation) @@ -144,49 +145,53 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with options: Map[String, String], isExternal: Boolean): Unit = { val (dbName, tblName) = processDatabaseAndTableName("default", tableName) - val tbl = new Table(dbName, tblName) - - tbl.setProperty("spark.sql.sources.provider", provider) + val tableProperties = new scala.collection.mutable.HashMap[String, String] + tableProperties.put("spark.sql.sources.provider", provider) if (userSpecifiedSchema.isDefined) { val threshold = hive.conf.schemaStringLengthThreshold val schemaJsonString = userSpecifiedSchema.get.json // Split the JSON string. val parts = schemaJsonString.grouped(threshold).toSeq - tbl.setProperty("spark.sql.sources.schema.numParts", parts.size.toString) + tableProperties.put("spark.sql.sources.schema.numParts", parts.size.toString) parts.zipWithIndex.foreach { case (part, index) => - tbl.setProperty(s"spark.sql.sources.schema.part.${index}", part) + tableProperties.put(s"spark.sql.sources.schema.part.${index}", part) } } - options.foreach { case (key, value) => tbl.setSerdeParam(key, value) } - if (isExternal) { - tbl.setProperty("EXTERNAL", "TRUE") - tbl.setTableType(TableType.EXTERNAL_TABLE) + val tableType = if (isExternal) { + tableProperties.put("EXTERNAL", "TRUE") + ExternalTable } else { - tbl.setProperty("EXTERNAL", "FALSE") - tbl.setTableType(TableType.MANAGED_TABLE) - } - - // create the table - synchronized { - client.createTable(tbl, false) - } + tableProperties.put("EXTERNAL", "FALSE") + ManagedTable + } + + client.createTable( + HiveTable( + specifiedDatabase = Option(dbName), + name = tblName, + schema = Seq.empty, + partitionColumns = Seq.empty, + tableType = tableType, + properties = tableProperties.toMap, + serdeProperties = options)) } - def hiveDefaultTableFilePath(tableName: String): String = synchronized { - val currentDatabase = client.getDatabase(hive.sessionState.getCurrentDatabase) - - hiveWarehouse.getTablePath(currentDatabase, tableName).toString + def hiveDefaultTableFilePath(tableName: String): String = { + // Code based on: hiveWarehouse.getTablePath(currentDatabase, tableName) + new Path( + new Path(client.getDatabase(client.currentDatabase).location), + tableName.toLowerCase).toString } - def tableExists(tableIdentifier: Seq[String]): Boolean = synchronized { + def tableExists(tableIdentifier: Seq[String]): Boolean = { val tableIdent = processTableIdentifier(tableIdentifier) val databaseName = tableIdent .lift(tableIdent.size - 2) - .getOrElse(hive.sessionState.getCurrentDatabase) + .getOrElse(client.currentDatabase) val tblName = tableIdent.last - client.getTable(databaseName, tblName, false) != null + client.getTableOption(databaseName, tblName).isDefined } def lookupRelation( @@ -194,18 +199,11 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with alias: Option[String]): LogicalPlan = { val tableIdent = processTableIdentifier(tableIdentifier) val databaseName = tableIdent.lift(tableIdent.size - 2).getOrElse( - hive.sessionState.getCurrentDatabase) + client.currentDatabase) val tblName = tableIdent.last - val table = try { - synchronized { - client.getTable(databaseName, tblName) - } - } catch { - case te: org.apache.hadoop.hive.ql.metadata.InvalidTableException => - throw new NoSuchTableException - } + val table = client.getTable(databaseName, tblName) - if (table.getProperty("spark.sql.sources.provider") != null) { + if (table.properties.get("spark.sql.sources.provider").isDefined) { val dataSourceTable = cachedDataSourceTables(QualifiedTableName(databaseName, tblName).toLowerCase) // Then, if alias is specified, wrap the table with a Subquery using the alias. @@ -215,22 +213,16 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with Subquery(tableIdent.last, dataSourceTable)) withAlias - } else if (table.isView) { - // if the unresolved relation is from hive view - // parse the text into logic node. - HiveQl.createPlanForView(table, alias) + } else if (table.tableType == VirtualView) { + val viewText = table.viewText.getOrElse(sys.error("Invalid view without text.")) + alias match { + // because hive use things like `_c0` to build the expanded text + // currently we cannot support view from "create view v1(c1) as ..." + case None => Subquery(table.name, HiveQl.createPlan(viewText)) + case Some(aliasText) => Subquery(aliasText, HiveQl.createPlan(viewText)) + } } else { - val partitions: Seq[Partition] = - if (table.isPartitioned) { - synchronized { - HiveShim.getAllPartitionsOf(client, table).toSeq - } - } else { - Nil - } - - MetastoreRelation(databaseName, tblName, alias)( - table.getTTable, partitions.map(part => part.getTPartition))(hive) + MetastoreRelation(databaseName, tblName, alias)(table)(hive) } } @@ -318,178 +310,10 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with result.newInstance() } - override def getTables(databaseName: Option[String]): Seq[(String, Boolean)] = synchronized { - val dbName = if (!caseSensitive) { - if (databaseName.isDefined) Some(databaseName.get.toLowerCase) else None - } else { - databaseName - } - val db = dbName.getOrElse(hive.sessionState.getCurrentDatabase) - - client.getAllTables(db).map(tableName => (tableName, false)) - } - - /** - * Create table with specified database, table name, table description and schema - * @param databaseName Database Name - * @param tableName Table Name - * @param schema Schema of the new table, if not specified, will use the schema - * specified in crtTbl - * @param allowExisting if true, ignore AlreadyExistsException - * @param desc CreateTableDesc object which contains the SerDe info. Currently - * we support most of the features except the bucket. - */ - def createTable( - databaseName: String, - tableName: String, - schema: Seq[Attribute], - allowExisting: Boolean = false, - desc: Option[CreateTableDesc] = None) { - val hconf = hive.hiveconf - - val (dbName, tblName) = processDatabaseAndTableName(databaseName, tableName) - val tbl = new Table(dbName, tblName) - - val crtTbl: CreateTableDesc = desc.getOrElse(null) - - // We should respect the passed in schema, unless it's not set - val hiveSchema: JList[FieldSchema] = if (schema == null || schema.isEmpty) { - crtTbl.getCols - } else { - schema.map(attr => new FieldSchema(attr.name, toMetastoreType(attr.dataType), null)) - } - tbl.setFields(hiveSchema) - - // Most of code are similar with the DDLTask.createTable() of Hive, - if (crtTbl != null && crtTbl.getTblProps() != null) { - tbl.getTTable().getParameters().putAll(crtTbl.getTblProps()) - } - - if (crtTbl != null && crtTbl.getPartCols() != null) { - tbl.setPartCols(crtTbl.getPartCols()) - } - - if (crtTbl != null && crtTbl.getStorageHandler() != null) { - tbl.setProperty( - org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.META_TABLE_STORAGE, - crtTbl.getStorageHandler()) - } - - /* - * We use LazySimpleSerDe by default. - * - * If the user didn't specify a SerDe, and any of the columns are not simple - * types, we will have to use DynamicSerDe instead. - */ - if (crtTbl == null || crtTbl.getSerName() == null) { - val storageHandler = tbl.getStorageHandler() - if (storageHandler == null) { - logInfo(s"Default to LazySimpleSerDe for table $dbName.$tblName") - tbl.setSerializationLib(classOf[LazySimpleSerDe].getName()) - - import org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - import org.apache.hadoop.io.Text - import org.apache.hadoop.mapred.TextInputFormat - - tbl.setInputFormatClass(classOf[TextInputFormat]) - tbl.setOutputFormatClass(classOf[HiveIgnoreKeyTextOutputFormat[Text, Text]]) - tbl.setSerializationLib("org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe") - } else { - val serDeClassName = storageHandler.getSerDeClass().getName() - logInfo(s"Use StorageHandler-supplied $serDeClassName for table $dbName.$tblName") - tbl.setSerializationLib(serDeClassName) - } - } else { - // let's validate that the serde exists - val serdeName = crtTbl.getSerName() - try { - val d = ReflectionUtils.newInstance(hconf.getClassByName(serdeName), hconf) - if (d != null) { - logDebug("Found class for $serdeName") - } - } catch { - case e: SerDeException => throw new HiveException("Cannot validate serde: " + serdeName, e) - } - tbl.setSerializationLib(serdeName) - } - - if (crtTbl != null && crtTbl.getFieldDelim() != null) { - tbl.setSerdeParam(serdeConstants.FIELD_DELIM, crtTbl.getFieldDelim()) - tbl.setSerdeParam(serdeConstants.SERIALIZATION_FORMAT, crtTbl.getFieldDelim()) - } - if (crtTbl != null && crtTbl.getFieldEscape() != null) { - tbl.setSerdeParam(serdeConstants.ESCAPE_CHAR, crtTbl.getFieldEscape()) - } - - if (crtTbl != null && crtTbl.getCollItemDelim() != null) { - tbl.setSerdeParam(serdeConstants.COLLECTION_DELIM, crtTbl.getCollItemDelim()) - } - if (crtTbl != null && crtTbl.getMapKeyDelim() != null) { - tbl.setSerdeParam(serdeConstants.MAPKEY_DELIM, crtTbl.getMapKeyDelim()) - } - if (crtTbl != null && crtTbl.getLineDelim() != null) { - tbl.setSerdeParam(serdeConstants.LINE_DELIM, crtTbl.getLineDelim()) - } - HiveShim.setTblNullFormat(crtTbl, tbl) - - if (crtTbl != null && crtTbl.getSerdeProps() != null) { - val iter = crtTbl.getSerdeProps().entrySet().iterator() - while (iter.hasNext()) { - val m = iter.next() - tbl.setSerdeParam(m.getKey(), m.getValue()) - } - } - - if (crtTbl != null && crtTbl.getComment() != null) { - tbl.setProperty("comment", crtTbl.getComment()) - } - - if (crtTbl != null && crtTbl.getLocation() != null) { - HiveShim.setLocation(tbl, crtTbl) - } - - if (crtTbl != null && crtTbl.getSkewedColNames() != null) { - tbl.setSkewedColNames(crtTbl.getSkewedColNames()) - } - if (crtTbl != null && crtTbl.getSkewedColValues() != null) { - tbl.setSkewedColValues(crtTbl.getSkewedColValues()) - } - - if (crtTbl != null) { - tbl.setStoredAsSubDirectories(crtTbl.isStoredAsSubDirectories()) - tbl.setInputFormatClass(crtTbl.getInputFormat()) - tbl.setOutputFormatClass(crtTbl.getOutputFormat()) - } - - tbl.getTTable().getSd().setInputFormat(tbl.getInputFormatClass().getName()) - tbl.getTTable().getSd().setOutputFormat(tbl.getOutputFormatClass().getName()) - - if (crtTbl != null && crtTbl.isExternal()) { - tbl.setProperty("EXTERNAL", "TRUE") - tbl.setTableType(TableType.EXTERNAL_TABLE) - } - - // set owner - try { - tbl.setOwner(hive.hiveconf.getUser) - } catch { - case e: IOException => throw new HiveException("Unable to get current user", e) - } - - // set create time - tbl.setCreateTime((System.currentTimeMillis() / 1000).asInstanceOf[Int]) - - // TODO add bucket support - // TODO set more info if Hive upgrade + override def getTables(databaseName: Option[String]): Seq[(String, Boolean)] = { + val db = databaseName.getOrElse(client.currentDatabase) - // create the table - synchronized { - try client.createTable(tbl, allowExisting) catch { - case e: org.apache.hadoop.hive.metastore.api.AlreadyExistsException - if allowExisting => // Do nothing - case e: Throwable => throw e - } - } + client.listTables(db).map(tableName => (tableName, false)) } protected def processDatabaseAndTableName( @@ -598,42 +422,11 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with // Wait until children are resolved. case p: LogicalPlan if !p.childrenResolved => p - // TODO extra is in type of ASTNode which means the logical plan is not resolved - // Need to think about how to implement the CreateTableAsSelect.resolved - case CreateTableAsSelect(db, tableName, child, allowExisting, Some(extra: ASTNode)) => - val (dbName, tblName) = processDatabaseAndTableName(db, tableName) - val databaseName = dbName.getOrElse(hive.sessionState.getCurrentDatabase) - - // Get the CreateTableDesc from Hive SemanticAnalyzer - val desc: Option[CreateTableDesc] = if (tableExists(Seq(databaseName, tblName))) { - None - } else { - val sa = new SemanticAnalyzer(hive.hiveconf) { - override def analyzeInternal(ast: ASTNode) { - // A hack to intercept the SemanticAnalyzer.analyzeInternal, - // to ignore the SELECT clause of the CTAS - val method = classOf[SemanticAnalyzer].getDeclaredMethod( - "analyzeCreateTable", classOf[ASTNode], classOf[QB]) - method.setAccessible(true) - method.invoke(this, ast, this.getQB) - } - } - - sa.analyze(extra, new Context(hive.hiveconf)) - Some(sa.getQB().getTableDesc) - } - - // Check if the query specifies file format or storage handler. - val hasStorageSpec = desc match { - case Some(crtTbl) => - crtTbl != null && (crtTbl.getSerName != null || crtTbl.getStorageHandler != null) - case None => false - } - - if (hive.convertCTAS && !hasStorageSpec) { + case CreateTableAsSelect(desc, child, allowExisting) => + if (hive.convertCTAS && !desc.serde.isDefined) { // Do the conversion when spark.sql.hive.convertCTAS is true and the query // does not specify any storage format (file format and storage handler). - if (dbName.isDefined) { + if (desc.specifiedDatabase.isDefined) { throw new AnalysisException( "Cannot specify database name in a CTAS statement " + "when spark.sql.hive.convertCTAS is set to true.") @@ -641,7 +434,7 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with val mode = if (allowExisting) SaveMode.Ignore else SaveMode.ErrorIfExists CreateTableUsingAsSelect( - tblName, + desc.name, hive.conf.defaultDataSourceName, temporary = false, mode, @@ -650,19 +443,19 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with ) } else { execution.CreateTableAsSelect( - databaseName, - tableName, + desc.copy( + specifiedDatabase = Option(desc.specifiedDatabase.getOrElse(client.currentDatabase))), child, - allowExisting, - desc) + allowExisting) } case p: LogicalPlan if p.resolved => p - case p @ CreateTableAsSelect(db, tableName, child, allowExisting, None) => - val (dbName, tblName) = processDatabaseAndTableName(db, tableName) + case p @ CreateTableAsSelect(desc, child, allowExisting) => + val (dbName, tblName) = processDatabaseAndTableName(desc.database, desc.name) + if (hive.convertCTAS) { - if (dbName.isDefined) { + if (desc.specifiedDatabase.isDefined) { throw new AnalysisException( "Cannot specify database name in a CTAS statement " + "when spark.sql.hive.convertCTAS is set to true.") @@ -678,13 +471,10 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with child ) } else { - val databaseName = dbName.getOrElse(hive.sessionState.getCurrentDatabase) execution.CreateTableAsSelect( - databaseName, - tableName, + desc, child, - allowExisting, - None) + allowExisting) } } } @@ -767,7 +557,7 @@ private[hive] case class InsertIntoHiveTable( private[hive] case class MetastoreRelation (databaseName: String, tableName: String, alias: Option[String]) - (val table: TTable, val partitions: Seq[TPartition]) + (val table: HiveTable) (@transient sqlContext: SQLContext) extends LeafNode with MultiInstanceRelation { @@ -786,16 +576,63 @@ private[hive] case class MetastoreRelation Objects.hashCode(databaseName, tableName, alias, output) } - // TODO: Can we use org.apache.hadoop.hive.ql.metadata.Table as the type of table and - // use org.apache.hadoop.hive.ql.metadata.Partition as the type of elements of partitions. - // Right now, using org.apache.hadoop.hive.ql.metadata.Table and - // org.apache.hadoop.hive.ql.metadata.Partition will cause a NotSerializableException - // which indicates the SerDe we used is not Serializable. + @transient val hiveQlTable: Table = { + // We start by constructing an API table as Hive performs several important transformations + // internally when converting an API table to a QL table. + val tTable = new org.apache.hadoop.hive.metastore.api.Table() + tTable.setTableName(table.name) + tTable.setDbName(table.database) + + val tableParameters = new java.util.HashMap[String, String]() + tTable.setParameters(tableParameters) + table.properties.foreach { case (k, v) => tableParameters.put(k, v) } + + tTable.setTableType(table.tableType.name) + + val sd = new org.apache.hadoop.hive.metastore.api.StorageDescriptor() + tTable.setSd(sd) + sd.setCols(table.schema.map(c => new FieldSchema(c.name, c.hiveType, c.comment))) + tTable.setPartitionKeys( + table.partitionColumns.map(c => new FieldSchema(c.name, c.hiveType, c.comment))) + + table.location.foreach(sd.setLocation) + table.inputFormat.foreach(sd.setInputFormat) + table.outputFormat.foreach(sd.setOutputFormat) + + val serdeInfo = new org.apache.hadoop.hive.metastore.api.SerDeInfo + sd.setSerdeInfo(serdeInfo) + table.serde.foreach(serdeInfo.setSerializationLib) + val serdeParameters = new java.util.HashMap[String, String]() + serdeInfo.setParameters(serdeParameters) + table.serdeProperties.foreach { case (k, v) => serdeParameters.put(k, v) } + + new Table(tTable) + } + + @transient val hiveQlPartitions: Seq[Partition] = table.getAllPartitions.map { p => + val tPartition = new org.apache.hadoop.hive.metastore.api.Partition + tPartition.setDbName(databaseName) + tPartition.setTableName(tableName) + tPartition.setValues(p.values) + + val sd = new org.apache.hadoop.hive.metastore.api.StorageDescriptor() + tPartition.setSd(sd) + sd.setCols(table.schema.map(c => new FieldSchema(c.name, c.hiveType, c.comment))) + + sd.setLocation(p.storage.location) + sd.setInputFormat(p.storage.inputFormat) + sd.setOutputFormat(p.storage.outputFormat) + + val serdeInfo = new org.apache.hadoop.hive.metastore.api.SerDeInfo + sd.setSerdeInfo(serdeInfo) + serdeInfo.setSerializationLib(p.storage.serde) - @transient val hiveQlTable: Table = new Table(table) + val serdeParameters = new java.util.HashMap[String, String]() + serdeInfo.setParameters(serdeParameters) + table.serdeProperties.foreach { case (k, v) => serdeParameters.put(k, v) } + p.storage.serdeProperties.foreach { case (k, v) => serdeParameters.put(k, v) } - @transient val hiveQlPartitions: Seq[Partition] = partitions.map { p => - new Partition(hiveQlTable, p) + new Partition(hiveQlTable, tPartition) } @transient override lazy val statistics: Statistics = Statistics( @@ -865,7 +702,7 @@ private[hive] case class MetastoreRelation val columnOrdinals = AttributeMap(attributes.zipWithIndex) override def newInstance(): MetastoreRelation = { - MetastoreRelation(databaseName, tableName, alias)(table, partitions)(sqlContext) + MetastoreRelation(databaseName, tableName, alias)(table)(sqlContext) } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index 6176aee25e7a4..f30b196734c40 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -38,6 +38,7 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.trees.CurrentOrigin import org.apache.spark.sql.execution.ExplainCommand import org.apache.spark.sql.sources.DescribeCommand +import org.apache.spark.sql.hive.client._ import org.apache.spark.sql.hive.execution.{HiveNativeCommand, DropTable, AnalyzeTable, HiveScriptIOSchema} import org.apache.spark.sql.types._ import org.apache.spark.util.random.RandomSampler @@ -50,7 +51,19 @@ import scala.collection.JavaConversions._ * back for Hive to execute natively. Will be replaced with a native command that contains the * cmd string. */ -private[hive] case object NativePlaceholder extends Command +private[hive] case object NativePlaceholder extends LogicalPlan { + override def children: Seq[LogicalPlan] = Seq.empty + override def output: Seq[Attribute] = Seq.empty +} + +case class CreateTableAsSelect( + tableDesc: HiveTable, + child: LogicalPlan, + allowExisting: Boolean) extends UnaryNode with Command { + + override def output: Seq[Attribute] = Seq.empty[Attribute] + override lazy val resolved: Boolean = tableDesc.specifiedDatabase.isDefined && childrenResolved +} /** Provides a mapping from HiveQL statements to catalyst logical plans and expression trees. */ private[hive] object HiveQl { @@ -78,16 +91,16 @@ private[hive] object HiveQl { "TOK_ALTERVIEW_DROPPARTS", "TOK_ALTERVIEW_PROPERTIES", "TOK_ALTERVIEW_RENAME", - + "TOK_CREATEDATABASE", "TOK_CREATEFUNCTION", "TOK_CREATEINDEX", "TOK_CREATEROLE", "TOK_CREATEVIEW", - + "TOK_DESCDATABASE", "TOK_DESCFUNCTION", - + "TOK_DROPDATABASE", "TOK_DROPFUNCTION", "TOK_DROPINDEX", @@ -95,22 +108,22 @@ private[hive] object HiveQl { "TOK_DROPTABLE_PROPERTIES", "TOK_DROPVIEW", "TOK_DROPVIEW_PROPERTIES", - + "TOK_EXPORT", - + "TOK_GRANT", "TOK_GRANT_ROLE", - + "TOK_IMPORT", - + "TOK_LOAD", - + "TOK_LOCKTABLE", - + "TOK_MSCK", - + "TOK_REVOKE", - + "TOK_SHOW_COMPACTIONS", "TOK_SHOW_CREATETABLE", "TOK_SHOW_GRANT", @@ -127,9 +140,9 @@ private[hive] object HiveQl { "TOK_SHOWINDEXES", "TOK_SHOWLOCKS", "TOK_SHOWPARTITIONS", - + "TOK_SWITCHDATABASE", - + "TOK_UNLOCKTABLE" ) @@ -259,6 +272,7 @@ private[hive] object HiveQl { case otherMessage => throw new AnalysisException(otherMessage) } + case e: MatchError => throw e case e: Exception => throw new AnalysisException(e.getMessage) case e: NotImplementedError => @@ -272,14 +286,6 @@ private[hive] object HiveQl { } } - /** Creates LogicalPlan for a given VIEW */ - def createPlanForView(view: Table, alias: Option[String]): Subquery = alias match { - // because hive use things like `_c0` to build the expanded text - // currently we cannot support view from "create view v1(c1) as ..." - case None => Subquery(view.getTableName, createPlan(view.getViewExpandedText)) - case Some(aliasText) => Subquery(aliasText, createPlan(view.getViewExpandedText)) - } - def parseDdl(ddl: String): Seq[Attribute] = { val tree = try { @@ -453,6 +459,14 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C (keys, bitmasks) } + protected def getProperties(node: Node): Seq[(String, String)] = node match { + case Token("TOK_TABLEPROPLIST", list) => + list.map { + case Token("TOK_TABLEPROPERTY", Token(key, Nil) :: Token(value, Nil) :: Nil) => + (unquoteString(key) -> unquoteString(value)) + } + } + protected def nodeToPlan(node: Node): LogicalPlan = node match { // Special drop table that also uncaches. case Token("TOK_DROPTABLE", @@ -562,7 +576,62 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C children) val (db, tableName) = extractDbNameTableName(tableNameParts) - CreateTableAsSelect(db, tableName, nodeToPlan(query), allowExisting != None, Some(node)) + var tableDesc = + HiveTable( + specifiedDatabase = db, + name = tableName, + schema = Seq.empty, + partitionColumns = Seq.empty, + properties = Map.empty, + serdeProperties = Map.empty, + tableType = ManagedTable, + location = None, + inputFormat = None, + outputFormat = None, + serde = None) + + // TODO: Handle all the cases here... + children.foreach { + case Token("TOK_TBLRCFILE", Nil) => + import org.apache.hadoop.hive.ql.io.{RCFileInputFormat, RCFileOutputFormat} + tableDesc = tableDesc.copy( + outputFormat = Option(classOf[RCFileOutputFormat].getName), + inputFormat = Option(classOf[RCFileInputFormat[_, _]].getName)) + + if (tableDesc.serde.isEmpty) { + tableDesc = tableDesc.copy( + serde = Option("org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe")) + } + case Token("TOK_TBLORCFILE", Nil) => + tableDesc = tableDesc.copy( + inputFormat = Option("org.apache.hadoop.hive.ql.io.orc.OrcInputFormat"), + outputFormat = Option("org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat"), + serde = Option("org.apache.hadoop.hive.ql.io.orc.OrcSerde")) + + case Token("TOK_TBLPARQUETFILE", Nil) => + tableDesc = tableDesc.copy( + inputFormat = Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"), + outputFormat = Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat"), + serde = Option("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe")) + + case Token("TOK_TABLESERIALIZER", + Token("TOK_SERDENAME", Token(serdeName, Nil) :: otherProps) :: Nil) => + tableDesc = tableDesc.copy(serde = Option(unquoteString(serdeName))) + + otherProps match { + case Token("TOK_TABLEPROPERTIES", list :: Nil) :: Nil => + tableDesc = tableDesc.copy( + serdeProperties = tableDesc.serdeProperties ++ getProperties(list)) + case Nil => + } + + case Token("TOK_TABLEPROPERTIES", list :: Nil) => + tableDesc = tableDesc.copy(properties = tableDesc.properties ++ getProperties(list)) + + case _ => + } + + CreateTableAsSelect(tableDesc, nodeToPlan(query), allowExisting != None) // If its not a "CREATE TABLE AS" like above then just pass it back to hive as a native command. case Token("TOK_CREATETABLE", _) => NativePlaceholder @@ -759,7 +828,7 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C case Token("TOK_CUBE_GROUPBY", children) => Cube(children.map(nodeToExpr), withLateralView, selectExpressions) case _ => sys.error("Expect WITH CUBE") - }), + }), Some(Project(selectExpressions, withLateralView))).flatten.head } @@ -1077,6 +1146,15 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C } protected val escapedIdentifier = "`([^`]+)`".r + protected val doubleQuotedString = "\"([^\"]+)\"".r + protected val singleQuotedString = "'([^']+)'".r + + protected def unquoteString(str: String) = str match { + case singleQuotedString(s) => s + case doubleQuotedString(s) => s + case other => other + } + /** Strips backticks from ident if present */ protected def cleanIdentifier(ident: String): String = ident match { case escapedIdentifier(i) => i diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala index e556c74ffb015..b69312f0f8717 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala @@ -32,6 +32,7 @@ import org.apache.hadoop.mapred.{FileInputFormat, InputFormat, JobConf} import org.apache.spark.SerializableWritable import org.apache.spark.broadcast.Broadcast +import org.apache.spark.Logging import org.apache.spark.rdd.{EmptyRDD, HadoopRDD, RDD, UnionRDD} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.types.DateUtils @@ -57,7 +58,7 @@ class HadoopTableReader( @transient relation: MetastoreRelation, @transient sc: HiveContext, @transient hiveExtraConf: HiveConf) - extends TableReader { + extends TableReader with Logging { // Hadoop honors "mapred.map.tasks" as hint, but will ignore when mapred.job.tracker is "local". // https://hadoop.apache.org/docs/r1.0.4/mapred-default.html @@ -78,7 +79,7 @@ class HadoopTableReader( makeRDDForTable( hiveTable, Class.forName( - relation.tableDesc.getSerdeClassName, true, sc.sessionState.getConf.getClassLoader) + relation.tableDesc.getSerdeClassName, true, Utils.getSparkClassLoader) .asInstanceOf[Class[Deserializer]], filterOpt = None) @@ -145,7 +146,7 @@ class HadoopTableReader( partitionToDeserializer: Map[HivePartition, Class[_ <: Deserializer]], filterOpt: Option[PathFilter]): RDD[Row] = { - + // SPARK-5068:get FileStatus and do the filtering locally when the path is not exists def verifyPartitionPath( partitionToDeserializer: Map[HivePartition, Class[_ <: Deserializer]]): @@ -288,7 +289,7 @@ class HadoopTableReader( } } -private[hive] object HadoopTableReader extends HiveInspectors { +private[hive] object HadoopTableReader extends HiveInspectors with Logging { /** * Curried. After given an argument for 'path', the resulting JobConf => Unit closure is used to * instantiate a HadoopRDD. @@ -329,6 +330,8 @@ private[hive] object HadoopTableReader extends HiveInspectors { tableDeser.getObjectInspector).asInstanceOf[StructObjectInspector] } + logDebug(soi.toString) + val (fieldRefs, fieldOrdinals) = nonPartitionKeyAttrs.map { case (attr, ordinal) => soi.getStructFieldRef(attr.name) -> ordinal }.unzip diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientInterface.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientInterface.scala index a863aa77cb7e0..0a1d761a52f88 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientInterface.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientInterface.scala @@ -17,30 +17,35 @@ package org.apache.spark.sql.hive.client +import java.io.PrintStream +import java.util.{Map => JMap} + import org.apache.spark.sql.catalyst.analysis.{NoSuchDatabaseException, NoSuchTableException} -case class HiveDatabase( +private[hive] case class HiveDatabase( name: String, location: String) -abstract class TableType { val name: String } -case object ExternalTable extends TableType { override val name = "EXTERNAL_TABLE" } -case object IndexTable extends TableType { override val name = "INDEX_TABLE" } -case object ManagedTable extends TableType { override val name = "MANAGED_TABLE" } -case object VirtualView extends TableType { override val name = "VIRTUAL_VIEW" } +private[hive] abstract class TableType { val name: String } +private[hive] case object ExternalTable extends TableType { override val name = "EXTERNAL_TABLE" } +private[hive] case object IndexTable extends TableType { override val name = "INDEX_TABLE" } +private[hive] case object ManagedTable extends TableType { override val name = "MANAGED_TABLE" } +private[hive] case object VirtualView extends TableType { override val name = "VIRTUAL_VIEW" } -case class HiveStorageDescriptor( +// TODO: Use this for Tables and Partitions +private[hive] case class HiveStorageDescriptor( location: String, inputFormat: String, outputFormat: String, - serde: String) + serde: String, + serdeProperties: Map[String, String]) -case class HivePartition( +private[hive] case class HivePartition( values: Seq[String], storage: HiveStorageDescriptor) -case class HiveColumn(name: String, hiveType: String, comment: String) -case class HiveTable( +private[hive] case class HiveColumn(name: String, hiveType: String, comment: String) +private[hive] case class HiveTable( specifiedDatabase: Option[String], name: String, schema: Seq[HiveColumn], @@ -51,7 +56,8 @@ case class HiveTable( location: Option[String] = None, inputFormat: Option[String] = None, outputFormat: Option[String] = None, - serde: Option[String] = None) { + serde: Option[String] = None, + viewText: Option[String] = None) { @transient private[client] var client: ClientInterface = _ @@ -76,13 +82,17 @@ case class HiveTable( * internal and external classloaders for a given version of Hive and thus must expose only * shared classes. */ -trait ClientInterface { +private[hive] trait ClientInterface { /** * Runs a HiveQL command using Hive, returning the results as a list of strings. Each row will * result in one string. */ def runSqlHive(sql: String): Seq[String] + def setOut(stream: PrintStream): Unit + def setInfo(stream: PrintStream): Unit + def setError(stream: PrintStream): Unit + /** Returns the names of all tables in the given database. */ def listTables(dbName: String): Seq[String] @@ -114,6 +124,11 @@ trait ClientInterface { /** Creates a new database with the given name. */ def createDatabase(database: HiveDatabase): Unit + /** Returns the specified paritition or None if it does not exist. */ + def getPartitionOption( + hTable: HiveTable, + partitionSpec: JMap[String, String]): Option[HivePartition] + /** Returns all partitions for the given table. */ def getAllPartitions(hTable: HiveTable): Seq[HivePartition] diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala index ea52fea037f1f..6bca9d0179fe3 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.hive.client import java.io.{BufferedReader, InputStreamReader, File, PrintStream} import java.net.URI -import java.util.{ArrayList => JArrayList} +import java.util.{ArrayList => JArrayList, Map => JMap, List => JList, Set => JSet} import scala.collection.JavaConversions._ import scala.language.reflectiveCalls @@ -27,6 +27,7 @@ import scala.language.reflectiveCalls import org.apache.hadoop.fs.Path import org.apache.hadoop.hive.metastore.api.Database import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.metastore.TableType import org.apache.hadoop.hive.metastore.api import org.apache.hadoop.hive.metastore.api.FieldSchema import org.apache.hadoop.hive.ql.metadata @@ -54,19 +55,13 @@ import org.apache.spark.sql.execution.QueryExecutionException * @param config a collection of configuration options that will be added to the hive conf before * opening the hive client. */ -class ClientWrapper( +private[hive] class ClientWrapper( version: HiveVersion, config: Map[String, String]) extends ClientInterface with Logging with ReflectionMagic { - private val conf = new HiveConf(classOf[SessionState]) - config.foreach { case (k, v) => - logDebug(s"Hive Config: $k=$v") - conf.set(k, v) - } - // Circular buffer to hold what hive prints to STDOUT and ERR. Only printed when failures occur. private val outputBuffer = new java.io.OutputStream { var pos: Int = 0 @@ -99,17 +94,31 @@ class ClientWrapper( val original = Thread.currentThread().getContextClassLoader Thread.currentThread().setContextClassLoader(getClass.getClassLoader) val ret = try { - val newState = new SessionState(conf) - SessionState.start(newState) - newState.out = new PrintStream(outputBuffer, true, "UTF-8") - newState.err = new PrintStream(outputBuffer, true, "UTF-8") - newState + val oldState = SessionState.get() + if (oldState == null) { + val initialConf = new HiveConf(classOf[SessionState]) + config.foreach { case (k, v) => + logDebug(s"Hive Config: $k=$v") + initialConf.set(k, v) + } + val newState = new SessionState(initialConf) + SessionState.start(newState) + newState.out = new PrintStream(outputBuffer, true, "UTF-8") + newState.err = new PrintStream(outputBuffer, true, "UTF-8") + newState + } else { + oldState + } } finally { Thread.currentThread().setContextClassLoader(original) } ret } + /** Returns the configuration for the current session. */ + def conf: HiveConf = SessionState.get().getConf + + // TODO: should be a def?s private val client = Hive.get(conf) /** @@ -133,6 +142,18 @@ class ClientWrapper( ret } + def setOut(stream: PrintStream): Unit = withHiveState { + state.out = stream + } + + def setInfo(stream: PrintStream): Unit = withHiveState { + state.info = stream + } + + def setError(stream: PrintStream): Unit = withHiveState { + state.err = stream + } + override def currentDatabase: String = withHiveState { state.getCurrentDatabase } @@ -171,14 +192,20 @@ class ClientWrapper( partitionColumns = h.getPartCols.map(f => HiveColumn(f.getName, f.getType, f.getComment)), properties = h.getParameters.toMap, serdeProperties = h.getTTable.getSd.getSerdeInfo.getParameters.toMap, - tableType = ManagedTable, // TODO + tableType = h.getTableType match { + case TableType.MANAGED_TABLE => ManagedTable + case TableType.EXTERNAL_TABLE => ExternalTable + case TableType.VIRTUAL_VIEW => VirtualView + case TableType.INDEX_TABLE => IndexTable + }, location = version match { case hive.v12 => Option(h.call[URI]("getDataLocation")).map(_.toString) case hive.v13 => Option(h.call[Path]("getDataLocation")).map(_.toString) }, inputFormat = Option(h.getInputFormatClass).map(_.getName), outputFormat = Option(h.getOutputFormatClass).map(_.getName), - serde = Option(h.getSerializationLib)).withClient(this) + serde = Option(h.getSerializationLib), + viewText = Option(h.getViewExpandedText)).withClient(this) } converted } @@ -223,27 +250,40 @@ class ClientWrapper( client.alterTable(table.qualifiedName, qlTable) } + private def toHivePartition(partition: metadata.Partition): HivePartition = { + val apiPartition = partition.getTPartition + HivePartition( + values = Option(apiPartition.getValues).map(_.toSeq).getOrElse(Seq.empty), + storage = HiveStorageDescriptor( + location = apiPartition.getSd.getLocation, + inputFormat = apiPartition.getSd.getInputFormat, + outputFormat = apiPartition.getSd.getOutputFormat, + serde = apiPartition.getSd.getSerdeInfo.getSerializationLib, + serdeProperties = apiPartition.getSd.getSerdeInfo.getParameters.toMap)) + } + + override def getPartitionOption( + table: HiveTable, + partitionSpec: JMap[String, String]): Option[HivePartition] = withHiveState { + + val qlTable = toQlTable(table) + val qlPartition = client.getPartition(qlTable, partitionSpec, false) + Option(qlPartition).map(toHivePartition) + } + override def getAllPartitions(hTable: HiveTable): Seq[HivePartition] = withHiveState { val qlTable = toQlTable(hTable) val qlPartitions = version match { case hive.v12 => - client.call[metadata.Table, Set[metadata.Partition]]("getAllPartitionsForPruner", qlTable) + client.call[metadata.Table, JSet[metadata.Partition]]("getAllPartitionsForPruner", qlTable) case hive.v13 => - client.call[metadata.Table, Set[metadata.Partition]]("getAllPartitionsOf", qlTable) + client.call[metadata.Table, JSet[metadata.Partition]]("getAllPartitionsOf", qlTable) } - qlPartitions.map(_.getTPartition).map { p => - HivePartition( - values = Option(p.getValues).map(_.toSeq).getOrElse(Seq.empty), - storage = HiveStorageDescriptor( - location = p.getSd.getLocation, - inputFormat = p.getSd.getInputFormat, - outputFormat = p.getSd.getOutputFormat, - serde = p.getSd.getSerdeInfo.getSerializationLib)) - }.toSeq + qlPartitions.toSeq.map(toHivePartition) } override def listTables(dbName: String): Seq[String] = withHiveState { - client.getAllTables + client.getAllTables(dbName) } /** @@ -267,11 +307,12 @@ class ClientWrapper( try { val cmd_trimmed: String = cmd.trim() val tokens: Array[String] = cmd_trimmed.split("\\s+") + // The remainder of the command. val cmd_1: String = cmd_trimmed.substring(tokens(0).length()).trim() val proc: CommandProcessor = version match { case hive.v12 => classOf[CommandProcessorFactory] - .callStatic[String, HiveConf, CommandProcessor]("get", cmd_1, conf) + .callStatic[String, HiveConf, CommandProcessor]("get", tokens(0), conf) case hive.v13 => classOf[CommandProcessorFactory] .callStatic[Array[String], HiveConf, CommandProcessor]("get", Array(tokens(0)), conf) @@ -294,7 +335,7 @@ class ClientWrapper( res.toSeq case hive.v13 => val res = new JArrayList[Object] - driver.call[JArrayList[Object], Boolean]("getResults", res) + driver.call[JList[Object], Boolean]("getResults", res) res.map { r => r match { case s: String => s diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala index 710dbca6e3c66..7f94c93ba49c1 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.hive.client import java.io.File -import java.net.URLClassLoader +import java.net.{URL, URLClassLoader} import java.util import scala.language.reflectiveCalls @@ -30,9 +30,10 @@ import org.apache.spark.Logging import org.apache.spark.deploy.SparkSubmitUtils import org.apache.spark.sql.catalyst.util.quietly +import org.apache.spark.sql.hive.HiveContext /** Factory for `IsolatedClientLoader` with specific versions of hive. */ -object IsolatedClientLoader { +private[hive] object IsolatedClientLoader { /** * Creates isolated Hive client loaders by downloading the requested version from maven. */ @@ -49,7 +50,7 @@ object IsolatedClientLoader { case "13" | "0.13" | "0.13.0" | "0.13.1" => hive.v13 } - private def downloadVersion(version: HiveVersion): Seq[File] = { + private def downloadVersion(version: HiveVersion): Seq[URL] = { val hiveArtifacts = (Seq("hive-metastore", "hive-exec", "hive-common", "hive-serde") ++ (if (version.hasBuiltinsJar) "hive-builtins" :: Nil else Nil)) @@ -72,10 +73,10 @@ object IsolatedClientLoader { tempDir.mkdir() allFiles.foreach(f => FileUtils.copyFileToDirectory(f, tempDir)) - tempDir.listFiles() + tempDir.listFiles().map(_.toURL) } - private def resolvedVersions = new scala.collection.mutable.HashMap[HiveVersion, Seq[File]] + private def resolvedVersions = new scala.collection.mutable.HashMap[HiveVersion, Seq[URL]] } /** @@ -99,9 +100,9 @@ object IsolatedClientLoader { * @param baseClassLoader The spark classloader that is used to load shared classes. * */ -class IsolatedClientLoader( +private[hive] class IsolatedClientLoader( val version: HiveVersion, - val execJars: Seq[File] = Seq.empty, + val execJars: Seq[URL] = Seq.empty, val config: Map[String, String] = Map.empty, val isolationOn: Boolean = true, val rootClassLoader: ClassLoader = ClassLoader.getSystemClassLoader.getParent.getParent, @@ -112,7 +113,7 @@ class IsolatedClientLoader( assert(Try(baseClassLoader.loadClass("org.apache.hive.HiveConf")).isFailure) /** All jars used by the hive specific classloader. */ - protected def allJars = execJars.map(_.toURI.toURL).toArray + protected def allJars = execJars.toArray protected def isSharedClass(name: String): Boolean = name.contains("slf4j") || @@ -166,6 +167,12 @@ class IsolatedClientLoader( .getConstructors.head .newInstance(version, config) .asInstanceOf[ClientInterface] + } catch { + case ReflectionException(cnf: NoClassDefFoundError) => + throw new ClassNotFoundException( + s"$cnf when creating Hive client using classpath: ${execJars.mkString(", ")}\n" + + "Please make sure that jars for your version of hive and hadoop are included in the " + + s"paths passed to ${HiveContext.HIVE_METASTORE_JARS}.") } finally { Thread.currentThread.setContextClassLoader(baseClassLoader) } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ReflectionMagic.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ReflectionMagic.scala index 90d03049356b5..c600b158c5460 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ReflectionMagic.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ReflectionMagic.scala @@ -19,6 +19,14 @@ package org.apache.spark.sql.hive.client import scala.reflect._ +/** Unwraps reflection exceptions. */ +private[client] object ReflectionException { + def unapply(a: Throwable): Option[Throwable] = a match { + case ite: java.lang.reflect.InvocationTargetException => Option(ite.getCause) + case _ => None + } +} + /** * Provides implicit functions on any object for calling methods reflectively. */ diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala index 76a1965f3cb25..91e6ac4032204 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala @@ -24,8 +24,8 @@ import org.apache.spark.sql.SQLContext import org.apache.spark.sql.catalyst.expressions.Row import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoTable, LogicalPlan} import org.apache.spark.sql.execution.RunnableCommand -import org.apache.spark.sql.hive.HiveContext -import org.apache.spark.sql.hive.MetastoreRelation +import org.apache.spark.sql.hive.client.{HiveTable, HiveColumn} +import org.apache.spark.sql.hive.{HiveContext, MetastoreRelation, HiveMetastoreTypes} /** * Create table and insert the query result into it. @@ -39,17 +39,34 @@ import org.apache.spark.sql.hive.MetastoreRelation */ private[hive] case class CreateTableAsSelect( - database: String, - tableName: String, + tableDesc: HiveTable, query: LogicalPlan, - allowExisting: Boolean, - desc: Option[CreateTableDesc]) extends RunnableCommand { + allowExisting: Boolean) + extends RunnableCommand { + + def database: String = tableDesc.database + def tableName: String = tableDesc.name override def run(sqlContext: SQLContext): Seq[Row] = { val hiveContext = sqlContext.asInstanceOf[HiveContext] lazy val metastoreRelation: MetastoreRelation = { - // Create Hive Table - hiveContext.catalog.createTable(database, tableName, query.output, allowExisting, desc) + import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe + import org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + import org.apache.hadoop.io.Text + import org.apache.hadoop.mapred.TextInputFormat + + val withSchema = + tableDesc.copy( + schema = + query.output.map(c => + HiveColumn(c.name, HiveMetastoreTypes.toMetastoreType(c.dataType), null)), + inputFormat = + tableDesc.inputFormat.orElse(Some(classOf[TextInputFormat].getName)), + outputFormat = + tableDesc.outputFormat + .orElse(Some(classOf[HiveIgnoreKeyTextOutputFormat[Text, Text]].getName)), + serde = tableDesc.serde.orElse(Some(classOf[LazySimpleSerDe].getName()))) + hiveContext.catalog.client.createTable(withSchema) // Get the Metastore Relation hiveContext.catalog.lookupRelation(Seq(database, tableName), None) match { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala index 89995a91b1a92..de8954d5dec99 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala @@ -200,9 +200,7 @@ case class InsertIntoHiveTable( orderedPartitionSpec.put(entry.getName,partitionSpec.get(entry.getName).getOrElse("")) } val partVals = MetaStoreUtils.getPvals(table.hiveQlTable.getPartCols, partitionSpec) - catalog.synchronized { - catalog.client.validatePartitionNameCharacters(partVals) - } + // inheritTableSpecs is set to true. It should be set to false for a IMPORT query // which is currently considered as a Hive native command. val inheritTableSpecs = true @@ -211,7 +209,7 @@ case class InsertIntoHiveTable( if (numDynamicPartitions > 0) { catalog.synchronized { catalog.client.loadDynamicPartitions( - outputPath, + outputPath.toString, qualifiedTableName, orderedPartitionSpec, overwrite, @@ -224,31 +222,28 @@ case class InsertIntoHiveTable( // ifNotExists is only valid with static partition, refer to // https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DML#LanguageManualDML-InsertingdataintoHiveTablesfromqueries // scalastyle:on - val oldPart = catalog.synchronized { - catalog.client.getPartition( - catalog.client.getTable(qualifiedTableName), partitionSpec, false) - } - if (oldPart == null || !ifNotExists) { - catalog.synchronized { + val oldPart = + catalog.client.getPartitionOption( + catalog.client.getTable(table.databaseName, table.tableName), + partitionSpec) + + if (oldPart.isEmpty || !ifNotExists) { catalog.client.loadPartition( - outputPath, + outputPath.toString, qualifiedTableName, orderedPartitionSpec, overwrite, holdDDLTime, inheritTableSpecs, isSkewedStoreAsSubdir) - } } } } else { - catalog.synchronized { - catalog.client.loadTable( - outputPath, - qualifiedTableName, - overwrite, - holdDDLTime) - } + catalog.client.loadTable( + outputPath.toString, // TODO: URI + qualifiedTableName, + overwrite, + holdDDLTime) } // Invalidate the cache. diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala index a40a1e53117cd..abab1a223a43a 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala @@ -27,6 +27,7 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.RunnableCommand import org.apache.spark.sql.hive.HiveContext import org.apache.spark.sql.types._ +import org.apache.spark.util.Utils /** * Analyzes the given table in the current database to generate statistics, which will be @@ -84,8 +85,20 @@ case class AddJar(path: String) extends RunnableCommand { override def run(sqlContext: SQLContext): Seq[Row] = { val hiveContext = sqlContext.asInstanceOf[HiveContext] + val currentClassLoader = Utils.getContextOrSparkClassLoader + + // Add jar to current context + val jarURL = new java.io.File(path).toURL + val newClassLoader = new java.net.URLClassLoader(Array(jarURL), currentClassLoader) + Thread.currentThread.setContextClassLoader(newClassLoader) + org.apache.hadoop.hive.ql.metadata.Hive.get().getConf().setClassLoader(newClassLoader) + + // Add jar to isolated hive classloader hiveContext.runSqlHive(s"ADD JAR $path") + + // Add jar to executors hiveContext.sparkContext.addJar(path) + Seq(Row(0)) } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala index ca84b43a998b8..1f40a5340c2ce 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.hive.test import java.io.File import java.util.{Set => JavaSet} +import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.ql.exec.FunctionRegistry import org.apache.hadoop.hive.ql.io.avro.{AvroContainerInputFormat, AvroContainerOutputFormat} import org.apache.hadoop.hive.ql.metadata.Table @@ -62,6 +63,8 @@ object TestHive class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { self => + import HiveContext._ + // By clearing the port we force Spark to pick a new one. This allows us to rerun tests // without restarting the JVM. System.clearProperty("spark.hostPort") @@ -70,24 +73,16 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { hiveconf.set("hive.plan.serialization.format", "javaXML") lazy val warehousePath = Utils.createTempDir() - lazy val metastorePath = Utils.createTempDir() /** Sets up the system initially or after a RESET command */ - protected def configure(): Unit = { - warehousePath.delete() - metastorePath.delete() - setConf("javax.jdo.option.ConnectionURL", - s"jdbc:derby:;databaseName=$metastorePath;create=true") - setConf("hive.metastore.warehouse.dir", warehousePath.toString) - } + protected override def configure(): Map[String, String] = + newTemporaryConfiguration() ++ Map("hive.metastore.warehouse.dir" -> warehousePath.toString) val testTempDir = Utils.createTempDir() // For some hive test case which contain ${system:test.tmp.dir} System.setProperty("test.tmp.dir", testTempDir.getCanonicalPath) - configure() // Must be called before initializing the catalog below. - /** The location of the compiled hive distribution */ lazy val hiveHome = envVarToFile("HIVE_HOME") /** The location of the hive source code. */ @@ -195,6 +190,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { * A list of test tables and the DDL required to initialize them. A test table is loaded on * demand when a query are run against it. */ + @transient lazy val testTables = new mutable.HashMap[String, TestTable]() def registerTestTable(testTable: TestTable): Unit = { @@ -204,6 +200,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { // The test tables that are defined in the Hive QTestUtil. // /itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java // https://github.com/apache/hive/blob/branch-0.13/data/scripts/q_test_init.sql + @transient val hiveQTestUtilTables = Seq( TestTable("src", "CREATE TABLE src (key INT, value STRING)".cmd, @@ -236,16 +233,18 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { import org.apache.hadoop.mapred.{SequenceFileInputFormat, SequenceFileOutputFormat} import org.apache.thrift.protocol.TBinaryProtocol - val srcThrift = new Table("default", "src_thrift") - srcThrift.setFields(Nil) - srcThrift.setInputFormatClass(classOf[SequenceFileInputFormat[_,_]].getName) - // In Hive, SequenceFileOutputFormat will be substituted by HiveSequenceFileOutputFormat. - srcThrift.setOutputFormatClass(classOf[SequenceFileOutputFormat[_,_]].getName) - srcThrift.setSerializationLib(classOf[ThriftDeserializer].getName) - srcThrift.setSerdeParam("serialization.class", classOf[Complex].getName) - srcThrift.setSerdeParam("serialization.format", classOf[TBinaryProtocol].getName) - catalog.client.createTable(srcThrift) - + runSqlHive( + s""" + |CREATE TABLE src_thrift(fake INT) + |ROW FORMAT SERDE '${classOf[ThriftDeserializer].getName}' + |WITH SERDEPROPERTIES( + | 'serialization.class'='${classOf[Complex].getName}', + | 'serialization.format'='${classOf[TBinaryProtocol].getName}' + |) + |STORED AS + |INPUTFORMAT '${classOf[SequenceFileInputFormat[_,_]].getName}' + |OUTPUTFORMAT '${classOf[SequenceFileOutputFormat[_,_]].getName}' + """.stripMargin) runSqlHive( s"LOAD DATA LOCAL INPATH '${getHiveFile("data/files/complex.seq")}' INTO TABLE src_thrift") @@ -367,7 +366,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { if (!(loadedTables contains name)) { // Marks the table as loaded first to prevent infinite mutually recursive table loading. loadedTables += name - logInfo(s"Loading test table $name") + logDebug(s"Loading test table $name") val createCmds = testTables.get(name).map(_.commands).getOrElse(sys.error(s"Unknown test table $name")) createCmds.foreach(_()) @@ -384,9 +383,6 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { */ protected val originalUdfs: JavaSet[String] = FunctionRegistry.getFunctionNames - // Database default may not exist in 0.13.1, create it if not exist - HiveShim.createDefaultDBIfNeeded(this) - /** * Resets the test instance by deleting any tables that have been created. * TODO: also clear out UDFs, views, etc. @@ -401,24 +397,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { cacheManager.clearCache() loadedTables.clear() catalog.cachedDataSourceTables.invalidateAll() - catalog.client.getAllTables("default").foreach { t => - logDebug(s"Deleting table $t") - val table = catalog.client.getTable("default", t) - - catalog.client.getIndexes("default", t, 255).foreach { index => - catalog.client.dropIndex("default", t, index.getIndexName, true) - } - - if (!table.isIndexTable) { - catalog.client.dropTable("default", t) - } - } - - catalog.client.getAllDatabases.filterNot(_ == "default").foreach { db => - logDebug(s"Dropping Database: $db") - catalog.client.dropDatabase(db, true, false, true) - } - + catalog.client.reset() catalog.unregisterAllTables() FunctionRegistry.getFunctionNames.filterNot(originalUdfs.contains(_)).foreach { udfName => @@ -429,7 +408,8 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { hiveconf.set("fs.default.name", new File(".").toURI.toString) // It is important that we RESET first as broken hooks that might have been set could break // other sql exec here. - runSqlHive("RESET") + executionHive.runSqlHive("RESET") + metadataHive.runSqlHive("RESET") // For some reason, RESET does not reset the following variables... // https://issues.apache.org/jira/browse/HIVE-9004 runSqlHive("set hive.table.parameters.default=") @@ -437,7 +417,11 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { runSqlHive("set datanucleus.cache.collections.lazy=true") // Lots of tests fail if we do not change the partition whitelist from the default. runSqlHive("set hive.metastore.partition.name.whitelist.pattern=.*") - configure() + + configure().foreach { + case (k, v) => + metadataHive.runSqlHive(s"SET $k=$v") + } runSqlHive("USE default") diff --git a/sql/hive/src/test/resources/log4j.properties b/sql/hive/src/test/resources/log4j.properties index 5bc08062d30eb..92eaf1f2795ba 100644 --- a/sql/hive/src/test/resources/log4j.properties +++ b/sql/hive/src/test/resources/log4j.properties @@ -33,7 +33,7 @@ log4j.appender.FA.layout=org.apache.log4j.PatternLayout log4j.appender.FA.layout.ConversionPattern=%d{HH:mm:ss.SSS} %t %p %c{1}: %m%n # Set the logger level of File Appender to WARN -log4j.appender.FA.Threshold = INFO +log4j.appender.FA.Threshold = DEBUG # Some packages are noisy for no good reason. log4j.additivity.org.apache.hadoop.hive.serde2.lazy.LazyStruct=false diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ErrorPositionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ErrorPositionSuite.scala index d960a30e00738..30f5313d2b812 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ErrorPositionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ErrorPositionSuite.scala @@ -17,12 +17,11 @@ package org.apache.spark.sql.hive -import java.io.{OutputStream, PrintStream} - import scala.util.Try import org.scalatest.BeforeAndAfter +import org.apache.spark.sql.catalyst.util.quietly import org.apache.spark.sql.hive.test.TestHive._ import org.apache.spark.sql.hive.test.TestHive.implicits._ import org.apache.spark.sql.{AnalysisException, QueryTest} @@ -109,25 +108,6 @@ class ErrorPositionSuite extends QueryTest with BeforeAndAfter { "SELECT 1 + array(1)", "1 + array") } - /** Hive can be very noisy, messing up the output of our tests. */ - private def quietly[A](f: => A): A = { - val origErr = System.err - val origOut = System.out - try { - System.setErr(new PrintStream(new OutputStream { - def write(b: Int) = {} - })) - System.setOut(new PrintStream(new OutputStream { - def write(b: Int) = {} - })) - - f - } finally { - System.setErr(origErr) - System.setOut(origOut) - } - } - /** * Creates a test that checks to see if the error thrown when analyzing a given query includes * the location of the given token in the query string. diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala index 0538aa203c5a0..47c60f651d14c 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala @@ -31,6 +31,7 @@ import org.apache.hadoop.mapred.InvalidInputException import org.apache.spark.sql._ import org.apache.spark.util.Utils import org.apache.spark.sql.types._ +import org.apache.spark.sql.hive.client.{HiveTable, ManagedTable} import org.apache.spark.sql.hive.test.TestHive._ import org.apache.spark.sql.hive.test.TestHive.implicits._ import org.apache.spark.sql.parquet.ParquetRelation2 @@ -686,16 +687,21 @@ class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach { test("SPARK-6655 still support a schema stored in spark.sql.sources.schema") { val tableName = "spark6655" val schema = StructType(StructField("int", IntegerType, true) :: Nil) - // Manually create the metadata in metastore. - val tbl = new Table("default", tableName) - tbl.setProperty("spark.sql.sources.provider", "json") - tbl.setProperty("spark.sql.sources.schema", schema.json) - tbl.setProperty("EXTERNAL", "FALSE") - tbl.setTableType(TableType.MANAGED_TABLE) - tbl.setSerdeParam("path", catalog.hiveDefaultTableFilePath(tableName)) - catalog.synchronized { - catalog.client.createTable(tbl) - } + + val hiveTable = HiveTable( + specifiedDatabase = Some("default"), + name = tableName, + schema = Seq.empty, + partitionColumns = Seq.empty, + properties = Map( + "spark.sql.sources.provider" -> "json", + "spark.sql.sources.schema" -> schema.json, + "EXTERNAL" -> "FALSE"), + tableType = ManagedTable, + serdeProperties = Map( + "path" -> catalog.hiveDefaultTableFilePath(tableName))) + + catalog.client.createTable(hiveTable) invalidateTable(tableName) val actualSchema = table(tableName).schema diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/SerializationSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/SerializationSuite.scala index d6ddd539d159d..8afe5459d4f1b 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/SerializationSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/SerializationSuite.scala @@ -26,8 +26,10 @@ import org.apache.spark.sql.hive.test.TestHive class SerializationSuite extends FunSuite { test("[SPARK-5840] HiveContext should be serializable") { - val hiveContext = new HiveContext(TestHive.sparkContext) + val hiveContext = TestHive hiveContext.hiveconf - new JavaSerializer(new SparkConf()).newInstance().serialize(hiveContext) + val serializer = new JavaSerializer(new SparkConf()).newInstance() + val bytes = serializer.serialize(hiveContext) + val deSer = serializer.deserialize[AnyRef](bytes) } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala index 81e77ba257bf1..321dc8d7322b8 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala @@ -22,9 +22,13 @@ import org.apache.spark.sql.catalyst.util.quietly import org.apache.spark.util.Utils import org.scalatest.FunSuite +/** + * A simple set of tests that call the methods of a hive ClientInterface, loading different version + * of hive from maven central. These tests are simple in that they are mostly just testing to make + * sure that reflective calls are not throwing NoSuchMethod error, but the actually functionallity + * is not fully tested. + */ class VersionsSuite extends FunSuite with Logging { - val testType = "derby" - private def buildConf() = { lazy val warehousePath = Utils.createTempDir() lazy val metastorePath = Utils.createTempDir() @@ -50,6 +54,14 @@ class VersionsSuite extends FunSuite with Logging { causes } + private val emptyDir = Utils.createTempDir().getCanonicalPath + + private def partSpec = { + val hashMap = new java.util.LinkedHashMap[String, String] + hashMap.put("key", "1") + hashMap + } + // Its actually pretty easy to mess things up and have all of your tests "pass" by accidentally // connecting to an auto-populated, in-process metastore. Let's make sure we are getting the // versions right by forcing a known compatibility failure. @@ -66,10 +78,9 @@ class VersionsSuite extends FunSuite with Logging { private var client: ClientInterface = null versions.foreach { version => - test(s"$version: listTables") { + test(s"$version: create client") { client = null client = IsolatedClientLoader.forVersion(version, buildConf()).client - client.listTables("default") } test(s"$version: createDatabase") { @@ -101,5 +112,64 @@ class VersionsSuite extends FunSuite with Logging { test(s"$version: getTable") { client.getTable("default", "src") } + + test(s"$version: listTables") { + assert(client.listTables("default") === Seq("src")) + } + + test(s"$version: currentDatabase") { + assert(client.currentDatabase === "default") + } + + test(s"$version: getDatabase") { + client.getDatabase("default") + } + + test(s"$version: alterTable") { + client.alterTable(client.getTable("default", "src")) + } + + test(s"$version: set command") { + client.runSqlHive("SET spark.sql.test.key=1") + } + + test(s"$version: create partitioned table DDL") { + client.runSqlHive("CREATE TABLE src_part (value INT) PARTITIONED BY (key INT)") + client.runSqlHive("ALTER TABLE src_part ADD PARTITION (key = '1')") + } + + test(s"$version: getPartitions") { + client.getAllPartitions(client.getTable("default", "src_part")) + } + + test(s"$version: loadPartition") { + client.loadPartition( + emptyDir, + "default.src_part", + partSpec, + false, + false, + false, + false) + } + + test(s"$version: loadTable") { + client.loadTable( + emptyDir, + "src", + false, + false) + } + + test(s"$version: loadDynamicPartitions") { + client.loadDynamicPartitions( + emptyDir, + "default.src_part", + partSpec, + false, + 1, + false, + false) + } } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala index a3eacbd4e3981..9c056e493bfde 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala @@ -300,6 +300,8 @@ abstract class HiveComparisonTest val hiveQueries = queryList.map(new TestHive.QueryExecution(_)) // Make sure we can at least parse everything before attempting hive execution. + // Note this must only look at the logical plan as we might not be able to analyze if + // other DDL has not been executed yet. hiveQueries.foreach(_.logical) val computedResults = (queryList.zipWithIndex, hiveQueries, hiveCacheFiles).zipped.map { case ((queryString, i), hiveQuery, cachedAnswerFile)=> diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index ac10b173307d8..7d728fe87bda7 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -900,7 +900,7 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { |DROP TABLE IF EXISTS dynamic_part_table; """.stripMargin) - test("Dynamic partition folder layout") { + ignore("Dynamic partition folder layout") { sql("DROP TABLE IF EXISTS dynamic_part_table") sql("CREATE TABLE dynamic_part_table(intcol INT) PARTITIONED BY (partcol1 INT, partcol2 INT)") sql("SET hive.exec.dynamic.partition.mode=nonstrict") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala index 45f10e2fe64aa..de6a41ce5bfcb 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala @@ -150,20 +150,21 @@ class PruningSuite extends HiveComparisonTest with BeforeAndAfter { val (actualScannedColumns, actualPartValues) = plan.collect { case p @ HiveTableScan(columns, relation, _) => val columnNames = columns.map(_.name) - val partValues = p.prunePartitions(relation.hiveQlPartitions).map(_.getValues) + val partValues = if (relation.table.isPartitioned) { + p.prunePartitions(relation.hiveQlPartitions).map(_.getValues) + } else { + Seq.empty + } (columnNames, partValues) }.head assert(actualOutputColumns === expectedOutputColumns, "Output columns mismatch") assert(actualScannedColumns === expectedScannedColumns, "Scanned columns mismatch") - assert( - actualPartValues.length === expectedPartValues.length, - "Partition value count mismatches") + val actualPartitions = actualPartValues.map(_.toSeq.mkString(",")).sorted + val expectedPartitions = expectedPartValues.map(_.mkString(",")).sorted - for ((actual, expected) <- actualPartValues.zip(expectedPartValues)) { - assert(actual sameElements expected, "Partition values mismatch") - } + assert(actualPartitions === expectedPartitions, "Partitions selected do not match") } // Creates a query test to compare query results generated by Hive and Catalyst. From 92f8f803a68e0c16771e9793098c6d76dfdf99af Mon Sep 17 00:00:00 2001 From: Shuo Xiang Date: Thu, 7 May 2015 20:55:08 -0700 Subject: [PATCH 53/75] [SPARK-7452] [MLLIB] fix bug in topBykey and update test the toArray function of the BoundedPriorityQueue does not necessarily preserve order. Add a counter-example as the test, which would fail the original impl. Author: Shuo Xiang Closes #5990 from coderxiang/topbykey-test and squashes the following commits: 98804c9 [Shuo Xiang] fix bug in topBykey and update test --- .../org/apache/spark/mllib/rdd/MLPairRDDFunctions.scala | 2 +- .../apache/spark/mllib/rdd/MLPairRDDFunctionsSuite.scala | 9 +++++---- 2 files changed, 6 insertions(+), 5 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/rdd/MLPairRDDFunctions.scala b/mllib/src/main/scala/org/apache/spark/mllib/rdd/MLPairRDDFunctions.scala index 5af55aaf84802..1b93e2d764c69 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/rdd/MLPairRDDFunctions.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/rdd/MLPairRDDFunctions.scala @@ -46,7 +46,7 @@ class MLPairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) extends Se combOp = (queue1, queue2) => { queue1 ++= queue2 } - ).mapValues(_.toArray.reverse) // This is an min-heap, so we reverse the order. + ).mapValues(_.toArray.sorted(ord.reverse)) // This is an min-heap, so we reverse the order. } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/rdd/MLPairRDDFunctionsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/rdd/MLPairRDDFunctionsSuite.scala index cb8fe4dba96f5..57216e8eb4a55 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/rdd/MLPairRDDFunctionsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/rdd/MLPairRDDFunctionsSuite.scala @@ -24,13 +24,14 @@ import org.apache.spark.mllib.rdd.MLPairRDDFunctions._ class MLPairRDDFunctionsSuite extends FunSuite with MLlibTestSparkContext { test("topByKey") { - val topMap = sc.parallelize(Array((1, 1), (1, 2), (3, 2), (3, 7), (5, 1), (3, 5)), 2) - .topByKey(2) + val topMap = sc.parallelize(Array((1, 7), (1, 3), (1, 6), (1, 1), (1, 2), (3, 2), (3, 7), (5, + 1), (3, 5)), 2) + .topByKey(5) .collectAsMap() assert(topMap.size === 3) - assert(topMap(1) === Array(2, 1)) - assert(topMap(3) === Array(7, 5)) + assert(topMap(1) === Array(7, 6, 3, 2, 1)) + assert(topMap(3) === Array(7, 5, 2)) assert(topMap(5) === Array(1)) } } From 3af423c92f117b5dd4dc6832dc50911cedb29abc Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Thu, 7 May 2015 20:59:42 -0700 Subject: [PATCH 54/75] [SPARK-6986] [SQL] Use Serializer2 in more cases. With https://github.com/apache/spark/commit/0a2b15ce43cf6096e1a7ae060b7c8a4010ce3b92, the serialization stream and deserialization stream has enough information to determine it is handling a key-value pari, a key, or a value. It is safe to use `SparkSqlSerializer2` in more cases. Author: Yin Huai Closes #5849 from yhuai/serializer2MoreCases and squashes the following commits: 53a5eaa [Yin Huai] Josh's comments. 487f540 [Yin Huai] Use BufferedOutputStream. 8385f95 [Yin Huai] Always create a new row at the deserialization side to work with sort merge join. c7e2129 [Yin Huai] Update tests. 4513d13 [Yin Huai] Use Serializer2 in more places. --- .../apache/spark/sql/execution/Exchange.scala | 23 ++---- .../sql/execution/SparkSqlSerializer2.scala | 74 ++++++++++++------- .../execution/SparkSqlSerializer2Suite.scala | 30 ++++---- 3 files changed, 69 insertions(+), 58 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala index 5b2e46962cd3b..f0d54cd6cd94f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala @@ -84,18 +84,8 @@ case class Exchange( def serializer( keySchema: Array[DataType], valueSchema: Array[DataType], + hasKeyOrdering: Boolean, numPartitions: Int): Serializer = { - // In ExternalSorter's spillToMergeableFile function, key-value pairs are written out - // through write(key) and then write(value) instead of write((key, value)). Because - // SparkSqlSerializer2 assumes that objects passed in are Product2, we cannot safely use - // it when spillToMergeableFile in ExternalSorter will be used. - // So, we will not use SparkSqlSerializer2 when - // - Sort-based shuffle is enabled and the number of reducers (numPartitions) is greater - // then the bypassMergeThreshold; or - // - newOrdering is defined. - val cannotUseSqlSerializer2 = - (sortBasedShuffleOn && numPartitions > bypassMergeThreshold) || newOrdering.nonEmpty - // It is true when there is no field that needs to be write out. // For now, we will not use SparkSqlSerializer2 when noField is true. val noField = @@ -104,14 +94,13 @@ case class Exchange( val useSqlSerializer2 = child.sqlContext.conf.useSqlSerializer2 && // SparkSqlSerializer2 is enabled. - !cannotUseSqlSerializer2 && // Safe to use Serializer2. SparkSqlSerializer2.support(keySchema) && // The schema of key is supported. SparkSqlSerializer2.support(valueSchema) && // The schema of value is supported. !noField val serializer = if (useSqlSerializer2) { logInfo("Using SparkSqlSerializer2.") - new SparkSqlSerializer2(keySchema, valueSchema) + new SparkSqlSerializer2(keySchema, valueSchema, hasKeyOrdering) } else { logInfo("Using SparkSqlSerializer.") new SparkSqlSerializer(sparkConf) @@ -154,7 +143,8 @@ case class Exchange( } val keySchema = expressions.map(_.dataType).toArray val valueSchema = child.output.map(_.dataType).toArray - shuffled.setSerializer(serializer(keySchema, valueSchema, numPartitions)) + shuffled.setSerializer( + serializer(keySchema, valueSchema, newOrdering.nonEmpty, numPartitions)) shuffled.map(_._2) @@ -179,7 +169,8 @@ case class Exchange( new ShuffledRDD[Row, Null, Null](rdd, part) } val keySchema = child.output.map(_.dataType).toArray - shuffled.setSerializer(serializer(keySchema, null, numPartitions)) + shuffled.setSerializer( + serializer(keySchema, null, newOrdering.nonEmpty, numPartitions)) shuffled.map(_._1) @@ -199,7 +190,7 @@ case class Exchange( val partitioner = new HashPartitioner(1) val shuffled = new ShuffledRDD[Null, Row, Row](rdd, partitioner) val valueSchema = child.output.map(_.dataType).toArray - shuffled.setSerializer(serializer(null, valueSchema, 1)) + shuffled.setSerializer(serializer(null, valueSchema, false, 1)) shuffled.map(_._2) case _ => sys.error(s"Exchange not implemented for $newPartitioning") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer2.scala index 35ad987eb1a63..256d527d7b636 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer2.scala @@ -27,7 +27,7 @@ import scala.reflect.ClassTag import org.apache.spark.serializer._ import org.apache.spark.Logging import org.apache.spark.sql.Row -import org.apache.spark.sql.catalyst.expressions.SpecificMutableRow +import org.apache.spark.sql.catalyst.expressions.{SpecificMutableRow, MutableRow, GenericMutableRow} import org.apache.spark.sql.types._ /** @@ -49,9 +49,9 @@ private[sql] class Serializer2SerializationStream( out: OutputStream) extends SerializationStream with Logging { - val rowOut = new DataOutputStream(out) - val writeKeyFunc = SparkSqlSerializer2.createSerializationFunction(keySchema, rowOut) - val writeValueFunc = SparkSqlSerializer2.createSerializationFunction(valueSchema, rowOut) + private val rowOut = new DataOutputStream(new BufferedOutputStream(out)) + private val writeKeyFunc = SparkSqlSerializer2.createSerializationFunction(keySchema, rowOut) + private val writeValueFunc = SparkSqlSerializer2.createSerializationFunction(valueSchema, rowOut) override def writeObject[T: ClassTag](t: T): SerializationStream = { val kv = t.asInstanceOf[Product2[Row, Row]] @@ -86,31 +86,44 @@ private[sql] class Serializer2SerializationStream( private[sql] class Serializer2DeserializationStream( keySchema: Array[DataType], valueSchema: Array[DataType], + hasKeyOrdering: Boolean, in: InputStream) extends DeserializationStream with Logging { - val rowIn = new DataInputStream(new BufferedInputStream(in)) + private val rowIn = new DataInputStream(new BufferedInputStream(in)) + + private def rowGenerator(schema: Array[DataType]): () => (MutableRow) = { + if (schema == null) { + () => null + } else { + if (hasKeyOrdering) { + // We have key ordering specified in a ShuffledRDD, it is not safe to reuse a mutable row. + () => new GenericMutableRow(schema.length) + } else { + // It is safe to reuse the mutable row. + val mutableRow = new SpecificMutableRow(schema) + () => mutableRow + } + } + } - val key = if (keySchema != null) new SpecificMutableRow(keySchema) else null - val value = if (valueSchema != null) new SpecificMutableRow(valueSchema) else null - val readKeyFunc = SparkSqlSerializer2.createDeserializationFunction(keySchema, rowIn, key) - val readValueFunc = SparkSqlSerializer2.createDeserializationFunction(valueSchema, rowIn, value) + // Functions used to return rows for key and value. + private val getKey = rowGenerator(keySchema) + private val getValue = rowGenerator(valueSchema) + // Functions used to read a serialized row from the InputStream and deserialize it. + private val readKeyFunc = SparkSqlSerializer2.createDeserializationFunction(keySchema, rowIn) + private val readValueFunc = SparkSqlSerializer2.createDeserializationFunction(valueSchema, rowIn) override def readObject[T: ClassTag](): T = { - readKeyFunc() - readValueFunc() - - (key, value).asInstanceOf[T] + (readKeyFunc(getKey()), readValueFunc(getValue())).asInstanceOf[T] } override def readKey[T: ClassTag](): T = { - readKeyFunc() - key.asInstanceOf[T] + readKeyFunc(getKey()).asInstanceOf[T] } override def readValue[T: ClassTag](): T = { - readValueFunc() - value.asInstanceOf[T] + readValueFunc(getValue()).asInstanceOf[T] } override def close(): Unit = { @@ -118,9 +131,10 @@ private[sql] class Serializer2DeserializationStream( } } -private[sql] class ShuffleSerializerInstance( +private[sql] class SparkSqlSerializer2Instance( keySchema: Array[DataType], - valueSchema: Array[DataType]) + valueSchema: Array[DataType], + hasKeyOrdering: Boolean) extends SerializerInstance { def serialize[T: ClassTag](t: T): ByteBuffer = @@ -137,7 +151,7 @@ private[sql] class ShuffleSerializerInstance( } def deserializeStream(s: InputStream): DeserializationStream = { - new Serializer2DeserializationStream(keySchema, valueSchema, s) + new Serializer2DeserializationStream(keySchema, valueSchema, hasKeyOrdering, s) } } @@ -148,12 +162,16 @@ private[sql] class ShuffleSerializerInstance( * The schema of keys is represented by `keySchema` and that of values is represented by * `valueSchema`. */ -private[sql] class SparkSqlSerializer2(keySchema: Array[DataType], valueSchema: Array[DataType]) +private[sql] class SparkSqlSerializer2( + keySchema: Array[DataType], + valueSchema: Array[DataType], + hasKeyOrdering: Boolean) extends Serializer with Logging with Serializable{ - def newInstance(): SerializerInstance = new ShuffleSerializerInstance(keySchema, valueSchema) + def newInstance(): SerializerInstance = + new SparkSqlSerializer2Instance(keySchema, valueSchema, hasKeyOrdering) override def supportsRelocationOfSerializedObjects: Boolean = { // SparkSqlSerializer2 is stateless and writes no stream headers @@ -323,11 +341,11 @@ private[sql] object SparkSqlSerializer2 { */ def createDeserializationFunction( schema: Array[DataType], - in: DataInputStream, - mutableRow: SpecificMutableRow): () => Unit = { - () => { - // If the schema is null, the returned function does nothing when it get called. - if (schema != null) { + in: DataInputStream): (MutableRow) => Row = { + if (schema == null) { + (mutableRow: MutableRow) => null + } else { + (mutableRow: MutableRow) => { var i = 0 while (i < schema.length) { schema(i) match { @@ -440,6 +458,8 @@ private[sql] object SparkSqlSerializer2 { } i += 1 } + + mutableRow } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlSerializer2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlSerializer2Suite.scala index 27f063d73a9a9..15337c4045436 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlSerializer2Suite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlSerializer2Suite.scala @@ -148,6 +148,15 @@ abstract class SparkSqlSerializer2Suite extends QueryTest with BeforeAndAfterAll table("shuffle").collect()) } + test("key schema is null") { + val aggregations = allColumns.split(",").map(c => s"COUNT($c)").mkString(",") + val df = sql(s"SELECT $aggregations FROM shuffle") + checkSerializer(df.queryExecution.executedPlan, serializerClass) + checkAnswer( + df, + Row(1000, 1000, 0, 1000, 1000, 1000, 1000, 1000, 1000, 1000, 1000, 1000, 1000, 1000)) + } + test("value schema is null") { val df = sql(s"SELECT col0 FROM shuffle ORDER BY col0") checkSerializer(df.queryExecution.executedPlan, serializerClass) @@ -167,29 +176,20 @@ class SparkSqlSerializer2SortShuffleSuite extends SparkSqlSerializer2Suite { override def beforeAll(): Unit = { super.beforeAll() // Sort merge will not be triggered. - sql("set spark.sql.shuffle.partitions = 200") - } - - test("key schema is null") { - val aggregations = allColumns.split(",").map(c => s"COUNT($c)").mkString(",") - val df = sql(s"SELECT $aggregations FROM shuffle") - checkSerializer(df.queryExecution.executedPlan, serializerClass) - checkAnswer( - df, - Row(1000, 1000, 0, 1000, 1000, 1000, 1000, 1000, 1000, 1000, 1000, 1000, 1000, 1000)) + val bypassMergeThreshold = + sparkContext.conf.getInt("spark.shuffle.sort.bypassMergeThreshold", 200) + sql(s"set spark.sql.shuffle.partitions=${bypassMergeThreshold-1}") } } /** For now, we will use SparkSqlSerializer for sort based shuffle with sort merge. */ class SparkSqlSerializer2SortMergeShuffleSuite extends SparkSqlSerializer2Suite { - // We are expecting SparkSqlSerializer. - override val serializerClass: Class[Serializer] = - classOf[SparkSqlSerializer].asInstanceOf[Class[Serializer]] - override def beforeAll(): Unit = { super.beforeAll() // To trigger the sort merge. - sql("set spark.sql.shuffle.partitions = 201") + val bypassMergeThreshold = + sparkContext.conf.getInt("spark.shuffle.sort.bypassMergeThreshold", 200) + sql(s"set spark.sql.shuffle.partitions=${bypassMergeThreshold + 1}") } } From 714db2ef52c0fe34418e252e5a6f220337022046 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 7 May 2015 22:32:13 -0700 Subject: [PATCH 55/75] [SPARK-7470] [SQL] Spark shell SQLContext crashes without hive This only happens if you have `SPARK_PREPEND_CLASSES` set. Then I built it with `build/sbt clean assembly compile` and just ran it with `bin/spark-shell`. ``` ... 15/05/07 17:07:30 INFO EventLoggingListener: Logging events to file:/tmp/spark-events/local-1431043649919 15/05/07 17:07:30 INFO SparkILoop: Created spark context.. Spark context available as sc. java.lang.NoClassDefFoundError: org/apache/hadoop/hive/conf/HiveConf at java.lang.Class.getDeclaredConstructors0(Native Method) at java.lang.Class.privateGetDeclaredConstructors(Class.java:2493) at java.lang.Class.getConstructor0(Class.java:2803) ... Caused by: java.lang.ClassNotFoundException: org.apache.hadoop.hive.conf.HiveConf at java.net.URLClassLoader$1.run(URLClassLoader.java:366) at java.net.URLClassLoader$1.run(URLClassLoader.java:355) at java.security.AccessController.doPrivileged(Native Method) at java.net.URLClassLoader.findClass(URLClassLoader.java:354) at java.lang.ClassLoader.loadClass(ClassLoader.java:425) at sun.misc.Launcher$AppClassLoader.loadClass(Launcher.java:308) at java.lang.ClassLoader.loadClass(ClassLoader.java:358) ... 52 more :10: error: not found: value sqlContext import sqlContext.implicits._ ^ :10: error: not found: value sqlContext import sqlContext.sql ^ ``` yhuai marmbrus Author: Andrew Or Closes #5997 from andrewor14/sql-shell-crash and squashes the following commits: 61147e6 [Andrew Or] Also expect NoClassDefFoundError --- .../src/main/scala/org/apache/spark/repl/SparkILoop.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala index 8dc0e0c965923..488f3a9f33256 100644 --- a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala +++ b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala @@ -1028,7 +1028,7 @@ class SparkILoop( logInfo("Created sql context (with Hive support)..") } catch { - case cnf: java.lang.ClassNotFoundException => + case _: java.lang.ClassNotFoundException | _: java.lang.NoClassDefFoundError => sqlContext = new SQLContext(sparkContext) logInfo("Created sql context..") } From f496bf3c539a873ffdf3aa803847ef7b50135bd7 Mon Sep 17 00:00:00 2001 From: wangfei Date: Thu, 7 May 2015 22:55:42 -0700 Subject: [PATCH 56/75] [SPARK-7232] [SQL] Add a Substitution batch for spark sql analyzer Added a new batch named `Substitution` before `Resolution` batch. The motivation for this is there are kind of cases we want to do some substitution on the parsed logical plan before resolve it. Consider this two cases: 1 CTE, for cte we first build a row logical plan ``` 'With Map(q1 -> 'Subquery q1 'Project ['key] 'UnresolvedRelation [src], None) 'Project [*] 'Filter ('key = 5) 'UnresolvedRelation [q1], None ``` In `With` logicalplan here is a map stored the (`q1-> subquery`), we want first take off the with command and substitute the `q1` of `UnresolvedRelation` by the `subquery` 2 Another example is Window function, in window function user may define some windows, we also need substitute the window name of child by the concrete window. this should also done in the Substitution batch. Author: wangfei Closes #5776 from scwf/addbatch and squashes the following commits: d4b962f [wangfei] added WindowsSubstitution 70f6932 [wangfei] Merge branch 'master' of https://github.com/apache/spark into addbatch ecaeafb [wangfei] address yhuai's comments 553005a [wangfei] fix test case 0c54798 [wangfei] address comments 29aaaaf [wangfei] fix compile 1c9a092 [wangfei] added Substitution bastch --- .../sql/catalyst/analysis/Analyzer.scala | 98 ++++++++++++------- 1 file changed, 60 insertions(+), 38 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 7e46ad851cdd3..bb7913e186a85 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -55,6 +55,10 @@ class Analyzer( val extendedResolutionRules: Seq[Rule[LogicalPlan]] = Nil lazy val batches: Seq[Batch] = Seq( + Batch("Substitution", fixedPoint, + CTESubstitution :: + WindowsSubstitution :: + Nil : _*), Batch("Resolution", fixedPoint, ResolveRelations :: ResolveReferences :: @@ -71,6 +75,55 @@ class Analyzer( extendedResolutionRules : _*) ) + /** + * Substitute child plan with cte definitions + */ + object CTESubstitution extends Rule[LogicalPlan] { + // TODO allow subquery to define CTE + def apply(plan: LogicalPlan): LogicalPlan = plan match { + case With(child, relations) => substituteCTE(child, relations) + case other => other + } + + def substituteCTE(plan: LogicalPlan, cteRelations: Map[String, LogicalPlan]): LogicalPlan = { + plan transform { + // In hive, if there is same table name in database and CTE definition, + // hive will use the table in database, not the CTE one. + // Taking into account the reasonableness and the implementation complexity, + // here use the CTE definition first, check table name only and ignore database name + // see https://github.com/apache/spark/pull/4929#discussion_r27186638 for more info + case u : UnresolvedRelation => + val substituted = cteRelations.get(u.tableIdentifier.last).map { relation => + val withAlias = u.alias.map(Subquery(_, relation)) + withAlias.getOrElse(relation) + } + substituted.getOrElse(u) + } + } + } + + /** + * Substitute child plan with WindowSpecDefinitions. + */ + object WindowsSubstitution extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + // Lookup WindowSpecDefinitions. This rule works with unresolved children. + case WithWindowDefinition(windowDefinitions, child) => + child.transform { + case plan => plan.transformExpressions { + case UnresolvedWindowExpression(c, WindowSpecReference(windowName)) => + val errorMessage = + s"Window specification $windowName is not defined in the WINDOW clause." + val windowSpecDefinition = + windowDefinitions + .get(windowName) + .getOrElse(failAnalysis(errorMessage)) + WindowExpression(c, windowSpecDefinition) + } + } + } + } + /** * Removes no-op Alias expressions from the plan. */ @@ -172,36 +225,20 @@ class Analyzer( * Replaces [[UnresolvedRelation]]s with concrete relations from the catalog. */ object ResolveRelations extends Rule[LogicalPlan] { - def getTable(u: UnresolvedRelation, cteRelations: Map[String, LogicalPlan]): LogicalPlan = { + def getTable(u: UnresolvedRelation): LogicalPlan = { try { - // In hive, if there is same table name in database and CTE definition, - // hive will use the table in database, not the CTE one. - // Taking into account the reasonableness and the implementation complexity, - // here use the CTE definition first, check table name only and ignore database name - cteRelations.get(u.tableIdentifier.last) - .map(relation => u.alias.map(Subquery(_, relation)).getOrElse(relation)) - .getOrElse(catalog.lookupRelation(u.tableIdentifier, u.alias)) + catalog.lookupRelation(u.tableIdentifier, u.alias) } catch { case _: NoSuchTableException => u.failAnalysis(s"no such table ${u.tableName}") } } - def apply(plan: LogicalPlan): LogicalPlan = { - val (realPlan, cteRelations) = plan match { - // TODO allow subquery to define CTE - // Add cte table to a temp relation map,drop `with` plan and keep its child - case With(child, relations) => (child, relations) - case other => (other, Map.empty[String, LogicalPlan]) - } - - realPlan transform { - case i@InsertIntoTable(u: UnresolvedRelation, _, _, _, _) => - i.copy( - table = EliminateSubQueries(getTable(u, cteRelations))) - case u: UnresolvedRelation => - getTable(u, cteRelations) - } + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + case i@InsertIntoTable(u: UnresolvedRelation, _, _, _, _) => + i.copy(table = EliminateSubQueries(getTable(u))) + case u: UnresolvedRelation => + getTable(u) } } @@ -664,21 +701,6 @@ class Analyzer( // We have to use transformDown at here to make sure the rule of // "Aggregate with Having clause" will be triggered. def apply(plan: LogicalPlan): LogicalPlan = plan transformDown { - // Lookup WindowSpecDefinitions. This rule works with unresolved children. - case WithWindowDefinition(windowDefinitions, child) => - child.transform { - case plan => plan.transformExpressions { - case UnresolvedWindowExpression(c, WindowSpecReference(windowName)) => - val errorMessage = - s"Window specification $windowName is not defined in the WINDOW clause." - val windowSpecDefinition = - windowDefinitions - .get(windowName) - .getOrElse(failAnalysis(errorMessage)) - WindowExpression(c, windowSpecDefinition) - } - } - // Aggregate with Having clause. This rule works with an unresolved Aggregate because // a resolved Aggregate will not have Window Functions. case f @ Filter(condition, a @ Aggregate(groupingExprs, aggregateExprs, child)) From c2f0821aad3b82dcd327e914c9b297e92526649d Mon Sep 17 00:00:00 2001 From: "Zhang, Liye" Date: Fri, 8 May 2015 09:10:58 +0100 Subject: [PATCH 57/75] [SPARK-7392] [CORE] bugfix: Kryo buffer size cannot be larger than 2M Author: Zhang, Liye Closes #5934 from liyezhang556520/kryoBufSize and squashes the following commits: 5707e04 [Zhang, Liye] fix import order 8693288 [Zhang, Liye] replace multiplier with ByteUnit methods 9bf93e9 [Zhang, Liye] add tests d91e5ed [Zhang, Liye] change kb to mb --- .../spark/serializer/KryoSerializer.scala | 11 +++---- .../serializer/KryoSerializerSuite.scala | 30 +++++++++++++++++++ 2 files changed, 36 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala index f9f78852f032b..64ba27f34d2f1 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -32,6 +32,7 @@ import org.apache.spark._ import org.apache.spark.api.python.PythonBroadcast import org.apache.spark.broadcast.HttpBroadcast import org.apache.spark.network.nio.{GetBlock, GotBlock, PutBlock} +import org.apache.spark.network.util.ByteUnit import org.apache.spark.scheduler.{CompressedMapStatus, HighlyCompressedMapStatus} import org.apache.spark.storage._ import org.apache.spark.util.BoundedPriorityQueue @@ -51,18 +52,18 @@ class KryoSerializer(conf: SparkConf) private val bufferSizeKb = conf.getSizeAsKb("spark.kryoserializer.buffer", "64k") - if (bufferSizeKb >= 2048) { + if (bufferSizeKb >= ByteUnit.GiB.toKiB(2)) { throw new IllegalArgumentException("spark.kryoserializer.buffer must be less than " + - s"2048 mb, got: + $bufferSizeKb mb.") + s"2048 mb, got: + ${ByteUnit.KiB.toMiB(bufferSizeKb)} mb.") } - private val bufferSize = (bufferSizeKb * 1024).toInt + private val bufferSize = ByteUnit.KiB.toBytes(bufferSizeKb).toInt val maxBufferSizeMb = conf.getSizeAsMb("spark.kryoserializer.buffer.max", "64m").toInt - if (maxBufferSizeMb >= 2048) { + if (maxBufferSizeMb >= ByteUnit.GiB.toMiB(2)) { throw new IllegalArgumentException("spark.kryoserializer.buffer.max must be less than " + s"2048 mb, got: + $maxBufferSizeMb mb.") } - private val maxBufferSize = maxBufferSizeMb * 1024 * 1024 + private val maxBufferSize = ByteUnit.MiB.toBytes(maxBufferSizeMb).toInt private val referenceTracking = conf.getBoolean("spark.kryo.referenceTracking", true) private val registrationRequired = conf.getBoolean("spark.kryo.registrationRequired", false) diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala index 778a7eee73b23..c7369de24b81f 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala @@ -32,6 +32,36 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext { conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") conf.set("spark.kryo.registrator", classOf[MyRegistrator].getName) + test("configuration limits") { + val conf1 = conf.clone() + val kryoBufferProperty = "spark.kryoserializer.buffer" + val kryoBufferMaxProperty = "spark.kryoserializer.buffer.max" + conf1.set(kryoBufferProperty, "64k") + conf1.set(kryoBufferMaxProperty, "64m") + new KryoSerializer(conf1).newInstance() + // 2048m = 2097152k + conf1.set(kryoBufferProperty, "2097151k") + conf1.set(kryoBufferMaxProperty, "64m") + // should not throw exception when kryoBufferMaxProperty < kryoBufferProperty + new KryoSerializer(conf1).newInstance() + conf1.set(kryoBufferMaxProperty, "2097151k") + new KryoSerializer(conf1).newInstance() + val conf2 = conf.clone() + conf2.set(kryoBufferProperty, "2048m") + val thrown1 = intercept[IllegalArgumentException](new KryoSerializer(conf2).newInstance()) + assert(thrown1.getMessage.contains(kryoBufferProperty)) + val conf3 = conf.clone() + conf3.set(kryoBufferMaxProperty, "2048m") + val thrown2 = intercept[IllegalArgumentException](new KryoSerializer(conf3).newInstance()) + assert(thrown2.getMessage.contains(kryoBufferMaxProperty)) + val conf4 = conf.clone() + conf4.set(kryoBufferProperty, "2g") + conf4.set(kryoBufferMaxProperty, "3g") + val thrown3 = intercept[IllegalArgumentException](new KryoSerializer(conf4).newInstance()) + assert(thrown3.getMessage.contains(kryoBufferProperty)) + assert(!thrown3.getMessage.contains(kryoBufferMaxProperty)) + } + test("basic types") { val ser = new KryoSerializer(conf).newInstance() def check[T: ClassTag](t: T) { From ebff7327af5efa9f57c605284de4fae6b050ae0f Mon Sep 17 00:00:00 2001 From: Lianhui Wang Date: Fri, 8 May 2015 08:44:46 -0500 Subject: [PATCH 58/75] [SPARK-6869] [PYSPARK] Add pyspark archives path to PYTHONPATH Based on https://github.com/apache/spark/pull/5478 that provide a PYSPARK_ARCHIVES_PATH env. within this PR, we just should export PYSPARK_ARCHIVES_PATH=/user/spark/pyspark.zip,/user/spark/python/lib/py4j-0.8.2.1-src.zip in conf/spark-env.sh when we don't install PySpark on each node of Yarn. i run python application successfully on yarn-client and yarn-cluster with this PR. andrewor14 sryza Sephiroth-Lin Can you take a look at this?thanks. Author: Lianhui Wang Closes #5580 from lianhuiwang/SPARK-6869 and squashes the following commits: 66ffa43 [Lianhui Wang] Update Client.scala c2ad0f9 [Lianhui Wang] Update Client.scala 1c8f664 [Lianhui Wang] Merge remote-tracking branch 'remotes/apache/master' into SPARK-6869 008850a [Lianhui Wang] Merge remote-tracking branch 'remotes/apache/master' into SPARK-6869 f0b4ed8 [Lianhui Wang] Merge remote-tracking branch 'remotes/apache/master' into SPARK-6869 150907b [Lianhui Wang] Merge remote-tracking branch 'remotes/apache/master' into SPARK-6869 20402cd [Lianhui Wang] use ZipEntry 9d87c3f [Lianhui Wang] update scala style e7bd971 [Lianhui Wang] address vanzin's comments 4b8a3ed [Lianhui Wang] use pyArchivesEnvOpt e6b573b [Lianhui Wang] address vanzin's comments f11f84a [Lianhui Wang] zip pyspark archives 5192cca [Lianhui Wang] update import path 3b1e4c8 [Lianhui Wang] address tgravescs's comments 9396346 [Lianhui Wang] put zip to make-distribution.sh 0d2baf7 [Lianhui Wang] update import paths e0179be [Lianhui Wang] add zip pyspark archives in build or sparksubmit 31e8e06 [Lianhui Wang] update code style 9f31dac [Lianhui Wang] update code and add comments f72987c [Lianhui Wang] add archives path to PYTHONPATH --- assembly/pom.xml | 21 ++++++++++ .../org/apache/spark/deploy/SparkSubmit.scala | 41 +++++++++++++++++++ project/SparkBuild.scala | 37 ++++++++++++++++- .../org/apache/spark/deploy/yarn/Client.scala | 23 ++++++++--- 4 files changed, 114 insertions(+), 8 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 2b4d0a990bf22..626c8577e31fe 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -92,6 +92,27 @@ true + + + org.apache.maven.plugins + maven-antrun-plugin + + + package + + run + + + + + + + + + + + + org.apache.maven.plugins diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 8a0327984e195..329fa06ba8ba5 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -332,6 +332,47 @@ object SparkSubmit { } } + // In yarn mode for a python app, add pyspark archives to files + // that can be distributed with the job + if (args.isPython && clusterManager == YARN) { + var pyArchives: String = null + val pyArchivesEnvOpt = sys.env.get("PYSPARK_ARCHIVES_PATH") + if (pyArchivesEnvOpt.isDefined) { + pyArchives = pyArchivesEnvOpt.get + } else { + if (!sys.env.contains("SPARK_HOME")) { + printErrorAndExit("SPARK_HOME does not exist for python application in yarn mode.") + } + val pythonPath = new ArrayBuffer[String] + for (sparkHome <- sys.env.get("SPARK_HOME")) { + val pyLibPath = Seq(sparkHome, "python", "lib").mkString(File.separator) + val pyArchivesFile = new File(pyLibPath, "pyspark.zip") + if (!pyArchivesFile.exists()) { + printErrorAndExit("pyspark.zip does not exist for python application in yarn mode.") + } + val py4jFile = new File(pyLibPath, "py4j-0.8.2.1-src.zip") + if (!py4jFile.exists()) { + printErrorAndExit("py4j-0.8.2.1-src.zip does not exist for python application " + + "in yarn mode.") + } + pythonPath += pyArchivesFile.getAbsolutePath() + pythonPath += py4jFile.getAbsolutePath() + } + pyArchives = pythonPath.mkString(",") + } + + pyArchives = pyArchives.split(",").map { localPath=> + val localURI = Utils.resolveURI(localPath) + if (localURI.getScheme != "local") { + args.files = mergeFileLists(args.files, localURI.toString) + new Path(localPath).getName + } else { + localURI.getPath + } + }.mkString(File.pathSeparator) + sysProps("spark.submit.pyArchives") = pyArchives + } + // If we're running a R app, set the main class to our specific R runner if (args.isR && deployMode == CLIENT) { if (args.primaryResource == SPARKR_SHELL) { diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 026855f8f6a5a..186345af0e60e 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -370,6 +370,7 @@ object Assembly { object PySparkAssembly { import sbtassembly.Plugin._ import AssemblyKeys._ + import java.util.zip.{ZipOutputStream, ZipEntry} lazy val settings = Seq( unmanagedJars in Compile += { BuildCommons.sparkHome / "python/lib/py4j-0.8.2.1-src.zip" }, @@ -377,16 +378,48 @@ object PySparkAssembly { // to be included in the assembly. We can't just add "python/" to the assembly's resource dir // list since that will copy unneeded / unwanted files. resourceGenerators in Compile <+= resourceManaged in Compile map { outDir: File => + val src = new File(BuildCommons.sparkHome, "python/pyspark") + + val zipFile = new File(BuildCommons.sparkHome , "python/lib/pyspark.zip") + zipFile.delete() + zipRecursive(src, zipFile) + val dst = new File(outDir, "pyspark") if (!dst.isDirectory()) { require(dst.mkdirs()) } - - val src = new File(BuildCommons.sparkHome, "python/pyspark") copy(src, dst) } ) + private def zipRecursive(source: File, destZipFile: File) = { + val destOutput = new ZipOutputStream(new FileOutputStream(destZipFile)) + addFilesToZipStream("", source, destOutput) + destOutput.flush() + destOutput.close() + } + + private def addFilesToZipStream(parent: String, source: File, output: ZipOutputStream): Unit = { + if (source.isDirectory()) { + output.putNextEntry(new ZipEntry(parent + source.getName())) + for (file <- source.listFiles()) { + addFilesToZipStream(parent + source.getName() + File.separator, file, output) + } + } else { + val in = new FileInputStream(source) + output.putNextEntry(new ZipEntry(parent + source.getName())) + val buf = new Array[Byte](8192) + var n = 0 + while (n != -1) { + n = in.read(buf) + if (n != -1) { + output.write(buf, 0, n) + } + } + in.close() + } + } + private def copy(src: File, dst: File): Seq[File] = { src.listFiles().flatMap { f => val child = new File(dst, f.getName()) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 20ecaf092e3f8..d21a7393478ce 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -468,6 +468,17 @@ private[spark] class Client( env("SPARK_YARN_USER_ENV") = userEnvs } + // if spark.submit.pyArchives is in sparkConf, append pyArchives to PYTHONPATH + // that can be passed on to the ApplicationMaster and the executors. + if (sparkConf.contains("spark.submit.pyArchives")) { + var pythonPath = sparkConf.get("spark.submit.pyArchives") + if (env.contains("PYTHONPATH")) { + pythonPath = Seq(env.get("PYTHONPATH"), pythonPath).mkString(File.pathSeparator) + } + env("PYTHONPATH") = pythonPath + sparkConf.setExecutorEnv("PYTHONPATH", pythonPath) + } + // In cluster mode, if the deprecated SPARK_JAVA_OPTS is set, we need to propagate it to // executors. But we can't just set spark.executor.extraJavaOptions, because the driver's // SparkContext will not let that set spark* system properties, which is expected behavior for @@ -1074,7 +1085,7 @@ object Client extends Logging { val hiveConf = hiveClass.getMethod("getConf").invoke(hive) val hiveConfClass = mirror.classLoader.loadClass("org.apache.hadoop.hive.conf.HiveConf") - val hiveConfGet = (param:String) => Option(hiveConfClass + val hiveConfGet = (param: String) => Option(hiveConfClass .getMethod("get", classOf[java.lang.String]) .invoke(hiveConf, param)) @@ -1096,7 +1107,7 @@ object Client extends Logging { val hive2Token = new Token[DelegationTokenIdentifier]() hive2Token.decodeFromUrlString(tokenStr) - credentials.addToken(new Text("hive.server2.delegation.token"),hive2Token) + credentials.addToken(new Text("hive.server2.delegation.token"), hive2Token) logDebug("Added hive.Server2.delegation.token to conf.") hiveClass.getMethod("closeCurrent").invoke(null) } else { @@ -1141,13 +1152,13 @@ object Client extends Logging { logInfo("Added HBase security token to credentials.") } catch { - case e:java.lang.NoSuchMethodException => + case e: java.lang.NoSuchMethodException => logInfo("HBase Method not found: " + e) - case e:java.lang.ClassNotFoundException => + case e: java.lang.ClassNotFoundException => logDebug("HBase Class not found: " + e) - case e:java.lang.NoClassDefFoundError => + case e: java.lang.NoClassDefFoundError => logDebug("HBase Class not found: " + e) - case e:Exception => + case e: Exception => logError("Exception when obtaining HBase security token: " + e) } } From c796be70f36e262b6a2ce75924bd970f40bf4045 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Fri, 8 May 2015 16:54:32 +0100 Subject: [PATCH 59/75] [SPARK-3454] separate json endpoints for data in the UI Exposes data available in the UI as json over http. Key points: * new endpoints, handled independently of existing XyzPage classes. Root entrypoint is `JsonRootResource` * Uses jersey + jackson for routing & converting POJOs into json * tests against known results in `HistoryServerSuite` * also fixes some minor issues w/ the UI -- synchronizing on access to `StorageListener` & `StorageStatusListener`, and fixing some inconsistencies w/ the way we handle retained jobs & stages. Author: Imran Rashid Closes #5940 from squito/SPARK-3454_better_test_files and squashes the following commits: 1a72ed6 [Imran Rashid] rats 85fdb3e [Imran Rashid] Merge branch 'no_php' into SPARK-3454 1fc65b0 [Imran Rashid] Revert "Revert "[SPARK-3454] separate json endpoints for data in the UI"" 1276900 [Imran Rashid] get rid of giant event file, replace w/ smaller one; check both shuffle read & shuffle write 4e12013 [Imran Rashid] just use test case name for expectation file name 863ef64 [Imran Rashid] rename json files to avoid strange file names and not look like php --- .rat-excludes | 7 + core/pom.xml | 8 + .../org/apache/spark/JobExecutionStatus.java | 8 +- .../status/api/v1/ApplicationStatus.java | 30 + .../spark/status/api/v1/StageStatus.java | 31 + .../spark/status/api/v1/TaskSorting.java | 48 + .../java/org/apache/spark/util/EnumUtil.java | 38 + .../scala/org/apache/spark/SparkContext.scala | 2 +- .../history/ApplicationHistoryProvider.scala | 4 +- .../deploy/history/FsHistoryProvider.scala | 14 +- .../spark/deploy/history/HistoryServer.scala | 20 +- .../spark/deploy/master/ApplicationInfo.scala | 2 +- .../apache/spark/deploy/master/Master.scala | 14 +- .../deploy/master/ui/ApplicationPage.scala | 19 +- .../spark/deploy/master/ui/MasterPage.scala | 12 +- .../spark/deploy/master/ui/MasterWebUI.scala | 24 +- .../spark/status/api/v1/AllJobsResource.scala | 98 ++ .../spark/status/api/v1/AllRDDResource.scala | 104 ++ .../status/api/v1/AllStagesResource.scala | 309 ++++ .../api/v1/ApplicationListResource.scala | 94 ++ .../status/api/v1/ExecutorListResource.scala | 36 + .../status/api/v1/JacksonMessageWriter.scala | 93 ++ .../status/api/v1/JsonRootResource.scala | 255 +++ .../api/v1/OneApplicationResource.scala | 31 + .../spark/status/api/v1/OneJobResource.scala | 41 + .../spark/status/api/v1/OneRDDResource.scala | 34 + .../status/api/v1/OneStageResource.scala | 150 ++ .../spark/status/api/v1/SecurityFilter.scala | 38 + .../spark/status/api/v1/SimpleDateParam.scala | 55 + .../org/apache/spark/status/api/v1/api.scala | 228 +++ .../spark/storage/StorageStatusListener.scala | 6 +- .../scala/org/apache/spark/ui/SparkUI.scala | 49 +- .../scala/org/apache/spark/ui/WebUI.scala | 8 +- .../apache/spark/ui/exec/ExecutorsPage.scala | 17 +- .../apache/spark/ui/jobs/AllJobsPage.scala | 14 +- .../apache/spark/ui/jobs/AllStagesPage.scala | 12 +- .../org/apache/spark/ui/jobs/JobPage.scala | 2 +- .../spark/ui/jobs/JobProgressListener.scala | 4 + .../org/apache/spark/ui/jobs/PoolPage.scala | 2 +- .../org/apache/spark/ui/jobs/StagePage.scala | 19 +- .../org/apache/spark/ui/storage/RDDPage.scala | 73 +- .../apache/spark/ui/storage/StoragePage.scala | 2 +- .../apache/spark/ui/storage/StorageTab.scala | 6 +- .../application_list_json_expectation.json | 53 + .../complete_stage_list_json_expectation.json | 67 + .../completed_app_list_json_expectation.json | 53 + .../executor_list_json_expectation.json | 17 + .../failed_stage_list_json_expectation.json | 23 + ...multi_attempt_app_json_1__expectation.json | 15 + ...multi_attempt_app_json_2__expectation.json | 15 + .../job_list_json_expectation.json | 43 + .../maxDate2_app_list_json_expectation.json | 10 + .../maxDate_app_list_json_expectation.json | 19 + .../minDate_app_list_json_expectation.json | 35 + .../one_app_json_expectation.json | 10 + ...ne_app_multi_attempt_json_expectation.json | 17 + .../one_job_json_expectation.json | 15 + .../one_rdd_storage_json_expectation.json | 64 + .../one_stage_attempt_json_expectation.json | 270 ++++ .../one_stage_json_expectation.json | 270 ++++ .../rdd_list_storage_json_expectation.json | 9 + .../running_app_list_json_expectation.json | 1 + .../stage_list_json_expectation.json | 89 ++ ...ist_with_accumulable_json_expectation.json | 27 + .../stage_task_list_expectation.json | 561 +++++++ ...multi_attempt_app_json_1__expectation.json | 193 +++ ...multi_attempt_app_json_2__expectation.json | 193 +++ ...k_list_w__offset___length_expectation.json | 1401 +++++++++++++++++ ...stage_task_list_w__sortBy_expectation.json | 561 +++++++ ...tBy_short_names___runtime_expectation.json | 561 +++++++ ...rtBy_short_names__runtime_expectation.json | 561 +++++++ ...mmary_w__custom_quantiles_expectation.json | 19 + ...sk_summary_w_shuffle_read_expectation.json | 19 + ...k_summary_w_shuffle_write_expectation.json | 19 + ...age_with_accumulable_json_expectation.json | 242 +++ ...eded_failed_job_list_json_expectation.json | 43 + .../succeeded_job_list_json_expectation.json | 29 + .../local-1422981759269/APPLICATION_COMPLETE | 0 .../local-1422981759269/EVENT_LOG_1 | 88 ++ .../local-1422981759269/SPARK_VERSION_1.2.0 | 0 .../local-1422981780767/APPLICATION_COMPLETE | 0 .../local-1422981780767/EVENT_LOG_1 | 82 + .../local-1422981780767/SPARK_VERSION_1.2.0 | 0 .../local-1425081759269/APPLICATION_COMPLETE | 0 .../local-1425081759269/EVENT_LOG_1 | 88 ++ .../local-1425081759269/SPARK_VERSION_1.2.0 | 0 .../local-1426533911241/APPLICATION_COMPLETE | 0 .../local-1426533911241/EVENT_LOG_1 | 24 + .../local-1426533911241/SPARK_VERSION_1.2.0 | 0 .../local-1426633911242/APPLICATION_COMPLETE | 0 .../local-1426633911242/EVENT_LOG_1 | 24 + .../local-1426633911242/SPARK_VERSION_1.2.0 | 0 .../spark-events/local-1430917381534 | 231 +++ .../org/apache/spark/JsonTestUtils.scala | 34 + .../spark/deploy/JsonProtocolSuite.scala | 14 +- .../deploy/history/HistoryServerSuite.scala | 231 ++- .../status/api/v1/SimpleDateParamTest.scala | 29 + .../org/apache/spark/ui/UISeleniumSuite.scala | 264 +++- docs/monitoring.md | 74 + pom.xml | 12 + 100 files changed, 8608 insertions(+), 172 deletions(-) create mode 100644 core/src/main/java/org/apache/spark/status/api/v1/ApplicationStatus.java create mode 100644 core/src/main/java/org/apache/spark/status/api/v1/StageStatus.java create mode 100644 core/src/main/java/org/apache/spark/status/api/v1/TaskSorting.java create mode 100644 core/src/main/java/org/apache/spark/util/EnumUtil.java create mode 100644 core/src/main/scala/org/apache/spark/status/api/v1/AllJobsResource.scala create mode 100644 core/src/main/scala/org/apache/spark/status/api/v1/AllRDDResource.scala create mode 100644 core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala create mode 100644 core/src/main/scala/org/apache/spark/status/api/v1/ApplicationListResource.scala create mode 100644 core/src/main/scala/org/apache/spark/status/api/v1/ExecutorListResource.scala create mode 100644 core/src/main/scala/org/apache/spark/status/api/v1/JacksonMessageWriter.scala create mode 100644 core/src/main/scala/org/apache/spark/status/api/v1/JsonRootResource.scala create mode 100644 core/src/main/scala/org/apache/spark/status/api/v1/OneApplicationResource.scala create mode 100644 core/src/main/scala/org/apache/spark/status/api/v1/OneJobResource.scala create mode 100644 core/src/main/scala/org/apache/spark/status/api/v1/OneRDDResource.scala create mode 100644 core/src/main/scala/org/apache/spark/status/api/v1/OneStageResource.scala create mode 100644 core/src/main/scala/org/apache/spark/status/api/v1/SecurityFilter.scala create mode 100644 core/src/main/scala/org/apache/spark/status/api/v1/SimpleDateParam.scala create mode 100644 core/src/main/scala/org/apache/spark/status/api/v1/api.scala create mode 100644 core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json create mode 100644 core/src/test/resources/HistoryServerExpectations/complete_stage_list_json_expectation.json create mode 100644 core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json create mode 100644 core/src/test/resources/HistoryServerExpectations/executor_list_json_expectation.json create mode 100644 core/src/test/resources/HistoryServerExpectations/failed_stage_list_json_expectation.json create mode 100644 core/src/test/resources/HistoryServerExpectations/job_list_from_multi_attempt_app_json_1__expectation.json create mode 100644 core/src/test/resources/HistoryServerExpectations/job_list_from_multi_attempt_app_json_2__expectation.json create mode 100644 core/src/test/resources/HistoryServerExpectations/job_list_json_expectation.json create mode 100644 core/src/test/resources/HistoryServerExpectations/maxDate2_app_list_json_expectation.json create mode 100644 core/src/test/resources/HistoryServerExpectations/maxDate_app_list_json_expectation.json create mode 100644 core/src/test/resources/HistoryServerExpectations/minDate_app_list_json_expectation.json create mode 100644 core/src/test/resources/HistoryServerExpectations/one_app_json_expectation.json create mode 100644 core/src/test/resources/HistoryServerExpectations/one_app_multi_attempt_json_expectation.json create mode 100644 core/src/test/resources/HistoryServerExpectations/one_job_json_expectation.json create mode 100644 core/src/test/resources/HistoryServerExpectations/one_rdd_storage_json_expectation.json create mode 100644 core/src/test/resources/HistoryServerExpectations/one_stage_attempt_json_expectation.json create mode 100644 core/src/test/resources/HistoryServerExpectations/one_stage_json_expectation.json create mode 100644 core/src/test/resources/HistoryServerExpectations/rdd_list_storage_json_expectation.json create mode 100644 core/src/test/resources/HistoryServerExpectations/running_app_list_json_expectation.json create mode 100644 core/src/test/resources/HistoryServerExpectations/stage_list_json_expectation.json create mode 100644 core/src/test/resources/HistoryServerExpectations/stage_list_with_accumulable_json_expectation.json create mode 100644 core/src/test/resources/HistoryServerExpectations/stage_task_list_expectation.json create mode 100644 core/src/test/resources/HistoryServerExpectations/stage_task_list_from_multi_attempt_app_json_1__expectation.json create mode 100644 core/src/test/resources/HistoryServerExpectations/stage_task_list_from_multi_attempt_app_json_2__expectation.json create mode 100644 core/src/test/resources/HistoryServerExpectations/stage_task_list_w__offset___length_expectation.json create mode 100644 core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_expectation.json create mode 100644 core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_short_names___runtime_expectation.json create mode 100644 core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_short_names__runtime_expectation.json create mode 100644 core/src/test/resources/HistoryServerExpectations/stage_task_summary_w__custom_quantiles_expectation.json create mode 100644 core/src/test/resources/HistoryServerExpectations/stage_task_summary_w_shuffle_read_expectation.json create mode 100644 core/src/test/resources/HistoryServerExpectations/stage_task_summary_w_shuffle_write_expectation.json create mode 100644 core/src/test/resources/HistoryServerExpectations/stage_with_accumulable_json_expectation.json create mode 100644 core/src/test/resources/HistoryServerExpectations/succeeded_failed_job_list_json_expectation.json create mode 100644 core/src/test/resources/HistoryServerExpectations/succeeded_job_list_json_expectation.json create mode 100755 core/src/test/resources/spark-events/local-1422981759269/APPLICATION_COMPLETE create mode 100755 core/src/test/resources/spark-events/local-1422981759269/EVENT_LOG_1 create mode 100755 core/src/test/resources/spark-events/local-1422981759269/SPARK_VERSION_1.2.0 create mode 100755 core/src/test/resources/spark-events/local-1422981780767/APPLICATION_COMPLETE create mode 100755 core/src/test/resources/spark-events/local-1422981780767/EVENT_LOG_1 create mode 100755 core/src/test/resources/spark-events/local-1422981780767/SPARK_VERSION_1.2.0 create mode 100755 core/src/test/resources/spark-events/local-1425081759269/APPLICATION_COMPLETE create mode 100755 core/src/test/resources/spark-events/local-1425081759269/EVENT_LOG_1 create mode 100755 core/src/test/resources/spark-events/local-1425081759269/SPARK_VERSION_1.2.0 create mode 100755 core/src/test/resources/spark-events/local-1426533911241/APPLICATION_COMPLETE create mode 100755 core/src/test/resources/spark-events/local-1426533911241/EVENT_LOG_1 create mode 100755 core/src/test/resources/spark-events/local-1426533911241/SPARK_VERSION_1.2.0 create mode 100755 core/src/test/resources/spark-events/local-1426633911242/APPLICATION_COMPLETE create mode 100755 core/src/test/resources/spark-events/local-1426633911242/EVENT_LOG_1 create mode 100755 core/src/test/resources/spark-events/local-1426633911242/SPARK_VERSION_1.2.0 create mode 100755 core/src/test/resources/spark-events/local-1430917381534 create mode 100644 core/src/test/scala/org/apache/spark/JsonTestUtils.scala create mode 100644 core/src/test/scala/org/apache/spark/status/api/v1/SimpleDateParamTest.scala diff --git a/.rat-excludes b/.rat-excludes index dccf2db8055ce..dc08c4a3673d2 100644 --- a/.rat-excludes +++ b/.rat-excludes @@ -74,5 +74,12 @@ logs .*scalastyle-output.xml .*dependency-reduced-pom.xml known_translations +json_expectation +local-1422981759269/* +local-1422981780767/* +local-1425081759269/* +local-1426533911241/* +local-1426633911242/* +local-1430917381534/* DESCRIPTION NAMESPACE diff --git a/core/pom.xml b/core/pom.xml index 164a836cb8f0b..fc42f48973fe9 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -228,6 +228,14 @@ json4s-jackson_${scala.binary.version} 3.2.10 + + com.sun.jersey + jersey-server + + + com.sun.jersey + jersey-core + org.apache.mesos mesos diff --git a/core/src/main/java/org/apache/spark/JobExecutionStatus.java b/core/src/main/java/org/apache/spark/JobExecutionStatus.java index 6e161313702bb..0287fb79f8dd2 100644 --- a/core/src/main/java/org/apache/spark/JobExecutionStatus.java +++ b/core/src/main/java/org/apache/spark/JobExecutionStatus.java @@ -17,9 +17,15 @@ package org.apache.spark; +import org.apache.spark.util.EnumUtil; + public enum JobExecutionStatus { RUNNING, SUCCEEDED, FAILED, - UNKNOWN + UNKNOWN; + + public static JobExecutionStatus fromString(String str) { + return EnumUtil.parseIgnoreCase(JobExecutionStatus.class, str); + } } diff --git a/core/src/main/java/org/apache/spark/status/api/v1/ApplicationStatus.java b/core/src/main/java/org/apache/spark/status/api/v1/ApplicationStatus.java new file mode 100644 index 0000000000000..8c7dcf776fda8 --- /dev/null +++ b/core/src/main/java/org/apache/spark/status/api/v1/ApplicationStatus.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.status.api.v1; + +import org.apache.spark.util.EnumUtil; + +public enum ApplicationStatus { + COMPLETED, + RUNNING; + + public static ApplicationStatus fromString(String str) { + return EnumUtil.parseIgnoreCase(ApplicationStatus.class, str); + } + +} diff --git a/core/src/main/java/org/apache/spark/status/api/v1/StageStatus.java b/core/src/main/java/org/apache/spark/status/api/v1/StageStatus.java new file mode 100644 index 0000000000000..9dbb565aab707 --- /dev/null +++ b/core/src/main/java/org/apache/spark/status/api/v1/StageStatus.java @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.status.api.v1; + +import org.apache.spark.util.EnumUtil; + +public enum StageStatus { + ACTIVE, + COMPLETE, + FAILED, + PENDING; + + public static StageStatus fromString(String str) { + return EnumUtil.parseIgnoreCase(StageStatus.class, str); + } +} diff --git a/core/src/main/java/org/apache/spark/status/api/v1/TaskSorting.java b/core/src/main/java/org/apache/spark/status/api/v1/TaskSorting.java new file mode 100644 index 0000000000000..f19ed01d5aebf --- /dev/null +++ b/core/src/main/java/org/apache/spark/status/api/v1/TaskSorting.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.status.api.v1; + +import org.apache.spark.util.EnumUtil; + +import java.util.HashSet; +import java.util.Set; + +public enum TaskSorting { + ID, + INCREASING_RUNTIME("runtime"), + DECREASING_RUNTIME("-runtime"); + + private final Set alternateNames; + private TaskSorting(String... names) { + alternateNames = new HashSet(); + for (String n: names) { + alternateNames.add(n); + } + } + + public static TaskSorting fromString(String str) { + String lower = str.toLowerCase(); + for (TaskSorting t: values()) { + if (t.alternateNames.contains(lower)) { + return t; + } + } + return EnumUtil.parseIgnoreCase(TaskSorting.class, str); + } + +} diff --git a/core/src/main/java/org/apache/spark/util/EnumUtil.java b/core/src/main/java/org/apache/spark/util/EnumUtil.java new file mode 100644 index 0000000000000..c40c7e727613c --- /dev/null +++ b/core/src/main/java/org/apache/spark/util/EnumUtil.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.util; + +import com.google.common.base.Joiner; +import org.apache.spark.annotation.Private; + +@Private +public class EnumUtil { + public static > E parseIgnoreCase(Class clz, String str) { + E[] constants = clz.getEnumConstants(); + if (str == null) { + return null; + } + for (E e : constants) { + if (e.name().equalsIgnoreCase(str)) { + return e; + } + } + throw new IllegalArgumentException( + String.format("Illegal type='%s'. Supported type values: %s", + str, Joiner.on(", ").join(constants))); + } +} diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index b5f040ceb15ca..b59f562d05ead 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -430,7 +430,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli _ui = if (conf.getBoolean("spark.ui.enabled", true)) { Some(SparkUI.createLiveUI(this, _conf, listenerBus, _jobProgressListener, - _env.securityManager,appName)) + _env.securityManager,appName, startTime = startTime)) } else { // For tests, do not enable the UI None diff --git a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala index 6a5011af17458..298a8201960d1 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala @@ -19,7 +19,7 @@ package org.apache.spark.deploy.history import org.apache.spark.ui.SparkUI -private[history] case class ApplicationAttemptInfo( +private[spark] case class ApplicationAttemptInfo( attemptId: Option[String], startTime: Long, endTime: Long, @@ -27,7 +27,7 @@ private[history] case class ApplicationAttemptInfo( sparkUser: String, completed: Boolean = false) -private[history] case class ApplicationHistoryInfo( +private[spark] case class ApplicationHistoryInfo( id: String, name: String, attempts: List[ApplicationAttemptInfo]) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index 993763f3aa092..45c2be34c8680 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -17,23 +17,21 @@ package org.apache.spark.deploy.history -import java.io.{IOException, BufferedInputStream, FileNotFoundException, InputStream} +import java.io.{BufferedInputStream, FileNotFoundException, IOException, InputStream} import java.util.concurrent.{ExecutorService, Executors, TimeUnit} import scala.collection.mutable -import scala.concurrent.duration.Duration -import com.google.common.util.concurrent.ThreadFactoryBuilder - -import com.google.common.util.concurrent.MoreExecutors -import org.apache.hadoop.fs.permission.AccessControlException +import com.google.common.util.concurrent.{MoreExecutors, ThreadFactoryBuilder} import org.apache.hadoop.fs.{FileStatus, Path} +import org.apache.hadoop.fs.permission.AccessControlException + +import org.apache.spark.{Logging, SecurityManager, SparkConf} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.io.CompressionCodec import org.apache.spark.scheduler._ import org.apache.spark.ui.SparkUI import org.apache.spark.util.{Clock, SystemClock, ThreadUtils, Utils} -import org.apache.spark.{Logging, SecurityManager, SparkConf} /** * A class that provides application history from event logs stored in the file system. @@ -151,7 +149,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) val conf = this.conf.clone() val appSecManager = new SecurityManager(conf) SparkUI.createHistoryUI(conf, replayBus, appSecManager, appId, - HistoryServer.getAttemptURI(appId, attempt.attemptId)) + HistoryServer.getAttemptURI(appId, attempt.attemptId), attempt.startTime) // Do not call ui.bind() to avoid creating a new server for each application } diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index 754c8e9b6668b..50522e69dc519 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -25,6 +25,7 @@ import org.eclipse.jetty.servlet.{ServletContextHandler, ServletHolder} import org.apache.spark.{Logging, SecurityManager, SparkConf} import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.status.api.v1.{ApplicationInfo, ApplicationsListResource, JsonRootResource, UIRoot} import org.apache.spark.ui.{SparkUI, UIUtils, WebUI} import org.apache.spark.ui.JettyUtils._ import org.apache.spark.util.{SignalLogger, Utils} @@ -45,7 +46,7 @@ class HistoryServer( provider: ApplicationHistoryProvider, securityManager: SecurityManager, port: Int) - extends WebUI(securityManager, port, conf) with Logging { + extends WebUI(securityManager, port, conf) with Logging with UIRoot { // How many applications to retain private val retainedApplications = conf.getInt("spark.history.retainedApplications", 50) @@ -56,7 +57,7 @@ class HistoryServer( require(parts.length == 1 || parts.length == 2, s"Invalid app key $key") val ui = provider .getAppUI(parts(0), if (parts.length > 1) Some(parts(1)) else None) - .getOrElse(throw new NoSuchElementException()) + .getOrElse(throw new NoSuchElementException(s"no app with key $key")) attachSparkUI(ui) ui } @@ -113,6 +114,10 @@ class HistoryServer( } } + def getSparkUI(appKey: String): Option[SparkUI] = { + Option(appCache.get(appKey)) + } + initialize() /** @@ -123,6 +128,9 @@ class HistoryServer( */ def initialize() { attachPage(new HistoryPage(this)) + + attachHandler(JsonRootResource.getJsonServlet(this)) + attachHandler(createStaticHandler(SparkUI.STATIC_RESOURCE_DIR, "/static")) val contextHandler = new ServletContextHandler @@ -160,7 +168,13 @@ class HistoryServer( * * @return List of all known applications. */ - def getApplicationList(): Iterable[ApplicationHistoryInfo] = provider.getListing() + def getApplicationList(): Iterable[ApplicationHistoryInfo] = { + provider.getListing() + } + + def getApplicationInfoList: Iterator[ApplicationInfo] = { + getApplicationList().iterator.map(ApplicationsListResource.appHistoryInfoToPublicAppInfo) + } /** * Returns the provider configuration to show in the listing page. diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala index f59d550d4f3b3..1620e95bea218 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala @@ -28,7 +28,7 @@ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.deploy.ApplicationDescription import org.apache.spark.util.Utils -private[deploy] class ApplicationInfo( +private[spark] class ApplicationInfo( val startTime: Long, val id: String, val desc: ApplicationDescription, diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 0fac3cdcf55e7..53e1903a3d125 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -754,9 +754,9 @@ private[master] class Master( /** * Rebuild a new SparkUI from the given application's event logs. - * Return whether this is successful. + * Return the UI if successful, else None */ - private def rebuildSparkUI(app: ApplicationInfo): Boolean = { + private[master] def rebuildSparkUI(app: ApplicationInfo): Option[SparkUI] = { val appName = app.desc.name val notFoundBasePath = HistoryServer.UI_PATH_PREFIX + "/not-found" try { @@ -764,7 +764,7 @@ private[master] class Master( .getOrElse { // Event logging is not enabled for this application app.desc.appUiUrl = notFoundBasePath - return false + return None } val eventLogFilePrefix = EventLoggingListener.getLogPath( @@ -787,7 +787,7 @@ private[master] class Master( val logInput = EventLoggingListener.openEventLog(new Path(eventLogFile), fs) val replayBus = new ReplayListenerBus() val ui = SparkUI.createHistoryUI(new SparkConf, replayBus, new SecurityManager(conf), - appName + status, HistoryServer.UI_PATH_PREFIX + s"/${app.id}") + appName + status, HistoryServer.UI_PATH_PREFIX + s"/${app.id}", app.startTime) val maybeTruncated = eventLogFile.endsWith(EventLoggingListener.IN_PROGRESS) try { replayBus.replay(logInput, eventLogFile, maybeTruncated) @@ -798,7 +798,7 @@ private[master] class Master( webUi.attachSparkUI(ui) // Application UI is successfully rebuilt, so link the Master UI to it app.desc.appUiUrl = ui.basePath - true + Some(ui) } catch { case fnf: FileNotFoundException => // Event logging is enabled for this application, but no event logs are found @@ -808,7 +808,7 @@ private[master] class Master( msg += " Did you specify the correct logging directory?" msg = URLEncoder.encode(msg, "UTF-8") app.desc.appUiUrl = notFoundBasePath + s"?msg=$msg&title=$title" - false + None case e: Exception => // Relay exception message to application UI page val title = s"Application history load error (${app.id})" @@ -817,7 +817,7 @@ private[master] class Master( logError(msg, e) msg = URLEncoder.encode(msg, "UTF-8") app.desc.appUiUrl = notFoundBasePath + s"?msg=$msg&exception=$exception&title=$title" - false + None } } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala index 273f077bd8f57..06e265f99e231 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala @@ -23,10 +23,8 @@ import scala.concurrent.Await import scala.xml.Node import akka.pattern.ask -import org.json4s.JValue -import org.json4s.JsonAST.JNothing -import org.apache.spark.deploy.{ExecutorState, JsonProtocol} +import org.apache.spark.deploy.ExecutorState import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState} import org.apache.spark.deploy.master.ExecutorDesc import org.apache.spark.ui.{UIUtils, WebUIPage} @@ -37,21 +35,6 @@ private[ui] class ApplicationPage(parent: MasterWebUI) extends WebUIPage("app") private val master = parent.masterActorRef private val timeout = parent.timeout - /** Executor details for a particular application */ - override def renderJson(request: HttpServletRequest): JValue = { - val appId = request.getParameter("appId") - val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterStateResponse] - val state = Await.result(stateFuture, timeout) - val app = state.activeApps.find(_.id == appId).getOrElse({ - state.completedApps.find(_.id == appId).getOrElse(null) - }) - if (app == null) { - JNothing - } else { - JsonProtocol.writeApplicationInfo(app) - } - } - /** Executor details for a particular application */ def render(request: HttpServletRequest): Seq[Node] = { val appId = request.getParameter("appId") diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala index 1f2c3fdbfb2bc..756927682cd24 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala @@ -35,10 +35,13 @@ private[ui] class MasterPage(parent: MasterWebUI) extends WebUIPage("") { private val master = parent.masterActorRef private val timeout = parent.timeout - override def renderJson(request: HttpServletRequest): JValue = { + def getMasterState: MasterStateResponse = { val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterStateResponse] - val state = Await.result(stateFuture, timeout) - JsonProtocol.writeMasterState(state) + Await.result(stateFuture, timeout) + } + + override def renderJson(request: HttpServletRequest): JValue = { + JsonProtocol.writeMasterState(getMasterState) } def handleAppKillRequest(request: HttpServletRequest): Unit = { @@ -68,8 +71,7 @@ private[ui] class MasterPage(parent: MasterWebUI) extends WebUIPage("") { /** Index view listing applications and executors */ def render(request: HttpServletRequest): Seq[Node] = { - val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterStateResponse] - val state = Await.result(stateFuture, timeout) + val state = getMasterState val workerHeaders = Seq("Worker Id", "Address", "State", "Cores", "Memory") val workers = state.workers.sortBy(_.id) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala index dea0a65eeeaa6..eb26e9f99c70b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala @@ -19,6 +19,7 @@ package org.apache.spark.deploy.master.ui import org.apache.spark.Logging import org.apache.spark.deploy.master.Master +import org.apache.spark.status.api.v1.{ApplicationsListResource, ApplicationInfo, JsonRootResource, UIRoot} import org.apache.spark.ui.{SparkUI, WebUI} import org.apache.spark.ui.JettyUtils._ import org.apache.spark.util.RpcUtils @@ -28,12 +29,15 @@ import org.apache.spark.util.RpcUtils */ private[master] class MasterWebUI(val master: Master, requestedPort: Int) - extends WebUI(master.securityMgr, requestedPort, master.conf, name = "MasterUI") with Logging { + extends WebUI(master.securityMgr, requestedPort, master.conf, name = "MasterUI") with Logging + with UIRoot { val masterActorRef = master.self val timeout = RpcUtils.askTimeout(master.conf) val killEnabled = master.conf.getBoolean("spark.ui.killEnabled", true) + val masterPage = new MasterPage(this) + initialize() /** Initialize all components of the server. */ @@ -43,6 +47,7 @@ class MasterWebUI(val master: Master, requestedPort: Int) attachPage(new HistoryNotFoundPage(this)) attachPage(masterPage) attachHandler(createStaticHandler(MasterWebUI.STATIC_RESOURCE_DIR, "/static")) + attachHandler(JsonRootResource.getJsonServlet(this)) attachHandler(createRedirectHandler( "/app/kill", "/", masterPage.handleAppKillRequest, httpMethods = Set("POST"))) attachHandler(createRedirectHandler( @@ -60,6 +65,23 @@ class MasterWebUI(val master: Master, requestedPort: Int) assert(serverInfo.isDefined, "Master UI must be bound to a server before detaching SparkUIs") ui.getHandlers.foreach(detachHandler) } + + def getApplicationInfoList: Iterator[ApplicationInfo] = { + val state = masterPage.getMasterState + val activeApps = state.activeApps.sortBy(_.startTime).reverse + val completedApps = state.completedApps.sortBy(_.endTime).reverse + activeApps.iterator.map { ApplicationsListResource.convertApplicationInfo(_, false) } ++ + completedApps.iterator.map { ApplicationsListResource.convertApplicationInfo(_, true) } + } + + def getSparkUI(appId: String): Option[SparkUI] = { + val state = masterPage.getMasterState + val activeApps = state.activeApps.sortBy(_.startTime).reverse + val completedApps = state.completedApps.sortBy(_.endTime).reverse + (activeApps ++ completedApps).find { _.id == appId }.flatMap { + master.rebuildSparkUI + } + } } private[master] object MasterWebUI { diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/AllJobsResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/AllJobsResource.scala new file mode 100644 index 0000000000000..5783df5d8220c --- /dev/null +++ b/core/src/main/scala/org/apache/spark/status/api/v1/AllJobsResource.scala @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.status.api.v1 + +import java.util.{Arrays, Date, List => JList} +import javax.ws.rs._ +import javax.ws.rs.core.MediaType + +import org.apache.spark.JobExecutionStatus +import org.apache.spark.ui.SparkUI +import org.apache.spark.ui.jobs.JobProgressListener +import org.apache.spark.ui.jobs.UIData.JobUIData + +@Produces(Array(MediaType.APPLICATION_JSON)) +private[v1] class AllJobsResource(ui: SparkUI) { + + @GET + def jobsList(@QueryParam("status") statuses: JList[JobExecutionStatus]): Seq[JobData] = { + val statusToJobs: Seq[(JobExecutionStatus, Seq[JobUIData])] = + AllJobsResource.getStatusToJobs(ui) + val adjStatuses: JList[JobExecutionStatus] = { + if (statuses.isEmpty) { + Arrays.asList(JobExecutionStatus.values(): _*) + } else { + statuses + } + } + val jobInfos = for { + (status, jobs) <- statusToJobs + job <- jobs if adjStatuses.contains(status) + } yield { + AllJobsResource.convertJobData(job, ui.jobProgressListener, false) + } + jobInfos.sortBy{- _.jobId} + } + +} + +private[v1] object AllJobsResource { + + def getStatusToJobs(ui: SparkUI): Seq[(JobExecutionStatus, Seq[JobUIData])] = { + val statusToJobs = ui.jobProgressListener.synchronized { + Seq( + JobExecutionStatus.RUNNING -> ui.jobProgressListener.activeJobs.values.toSeq, + JobExecutionStatus.SUCCEEDED -> ui.jobProgressListener.completedJobs.toSeq, + JobExecutionStatus.FAILED -> ui.jobProgressListener.failedJobs.reverse.toSeq + ) + } + statusToJobs + } + + def convertJobData( + job: JobUIData, + listener: JobProgressListener, + includeStageDetails: Boolean): JobData = { + listener.synchronized { + val lastStageInfo = listener.stageIdToInfo.get(job.stageIds.max) + val lastStageData = lastStageInfo.flatMap { s => + listener.stageIdToData.get((s.stageId, s.attemptId)) + } + val lastStageName = lastStageInfo.map { _.name }.getOrElse("(Unknown Stage Name)") + val lastStageDescription = lastStageData.flatMap { _.description } + new JobData( + jobId = job.jobId, + name = lastStageName, + description = lastStageDescription, + submissionTime = job.submissionTime.map{new Date(_)}, + completionTime = job.completionTime.map{new Date(_)}, + stageIds = job.stageIds, + jobGroup = job.jobGroup, + status = job.status, + numTasks = job.numTasks, + numActiveTasks = job.numActiveTasks, + numCompletedTasks = job.numCompletedTasks, + numSkippedTasks = job.numCompletedTasks, + numFailedTasks = job.numFailedTasks, + numActiveStages = job.numActiveStages, + numCompletedStages = job.completedStageIndices.size, + numSkippedStages = job.numSkippedStages, + numFailedStages = job.numFailedStages + ) + } + } +} diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/AllRDDResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/AllRDDResource.scala new file mode 100644 index 0000000000000..645ede26a0879 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/status/api/v1/AllRDDResource.scala @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.status.api.v1 + +import javax.ws.rs.{GET, Produces} +import javax.ws.rs.core.MediaType + +import org.apache.spark.storage.{RDDInfo, StorageStatus, StorageUtils} +import org.apache.spark.ui.SparkUI +import org.apache.spark.ui.storage.StorageListener + +@Produces(Array(MediaType.APPLICATION_JSON)) +private[v1] class AllRDDResource(ui: SparkUI) { + + @GET + def rddList(): Seq[RDDStorageInfo] = { + val storageStatusList = ui.storageListener.storageStatusList + val rddInfos = ui.storageListener.rddInfoList + rddInfos.map{rddInfo => + AllRDDResource.getRDDStorageInfo(rddInfo.id, rddInfo, storageStatusList, + includeDetails = false) + } + } + +} + +private[spark] object AllRDDResource { + + def getRDDStorageInfo( + rddId: Int, + listener: StorageListener, + includeDetails: Boolean): Option[RDDStorageInfo] = { + val storageStatusList = listener.storageStatusList + listener.rddInfoList.find { _.id == rddId }.map { rddInfo => + getRDDStorageInfo(rddId, rddInfo, storageStatusList, includeDetails) + } + } + + def getRDDStorageInfo( + rddId: Int, + rddInfo: RDDInfo, + storageStatusList: Seq[StorageStatus], + includeDetails: Boolean): RDDStorageInfo = { + val workers = storageStatusList.map { (rddId, _) } + val blockLocations = StorageUtils.getRddBlockLocations(rddId, storageStatusList) + val blocks = storageStatusList + .flatMap { _.rddBlocksById(rddId) } + .sortWith { _._1.name < _._1.name } + .map { case (blockId, status) => + (blockId, status, blockLocations.get(blockId).getOrElse(Seq[String]("Unknown"))) + } + + val dataDistribution = if (includeDetails) { + Some(storageStatusList.map { status => + new RDDDataDistribution( + address = status.blockManagerId.hostPort, + memoryUsed = status.memUsedByRdd(rddId), + memoryRemaining = status.memRemaining, + diskUsed = status.diskUsedByRdd(rddId) + ) } ) + } else { + None + } + val partitions = if (includeDetails) { + Some(blocks.map { case (id, block, locations) => + new RDDPartitionInfo( + blockName = id.name, + storageLevel = block.storageLevel.description, + memoryUsed = block.memSize, + diskUsed = block.diskSize, + executors = locations + ) + } ) + } else { + None + } + + new RDDStorageInfo( + id = rddId, + name = rddInfo.name, + numPartitions = rddInfo.numPartitions, + numCachedPartitions = rddInfo.numCachedPartitions, + storageLevel = rddInfo.storageLevel.description, + memoryUsed = rddInfo.memSize, + diskUsed = rddInfo.diskSize, + dataDistribution = dataDistribution, + partitions = partitions + ) + } +} diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala new file mode 100644 index 0000000000000..50608588f09ae --- /dev/null +++ b/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala @@ -0,0 +1,309 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.status.api.v1 + +import java.util.{Arrays, Date, List => JList} +import javax.ws.rs.{GET, PathParam, Produces, QueryParam} +import javax.ws.rs.core.MediaType + +import org.apache.spark.executor.{InputMetrics => InternalInputMetrics, OutputMetrics => InternalOutputMetrics, ShuffleReadMetrics => InternalShuffleReadMetrics, ShuffleWriteMetrics => InternalShuffleWriteMetrics, TaskMetrics => InternalTaskMetrics} +import org.apache.spark.scheduler.{AccumulableInfo => InternalAccumulableInfo, StageInfo} +import org.apache.spark.ui.SparkUI +import org.apache.spark.ui.jobs.UIData.{StageUIData, TaskUIData} +import org.apache.spark.util.Distribution + +@Produces(Array(MediaType.APPLICATION_JSON)) +private[v1] class AllStagesResource(ui: SparkUI) { + + @GET + def stageList(@QueryParam("status") statuses: JList[StageStatus]): Seq[StageData] = { + val listener = ui.jobProgressListener + val stageAndStatus = AllStagesResource.stagesAndStatus(ui) + val adjStatuses = { + if (statuses.isEmpty()) { + Arrays.asList(StageStatus.values(): _*) + } else { + statuses + } + } + for { + (status, stageList) <- stageAndStatus + stageInfo: StageInfo <- stageList if adjStatuses.contains(status) + stageUiData: StageUIData <- listener.synchronized { + listener.stageIdToData.get((stageInfo.stageId, stageInfo.attemptId)) + } + } yield { + AllStagesResource.stageUiToStageData(status, stageInfo, stageUiData, includeDetails = false) + } + } +} + +private[v1] object AllStagesResource { + def stageUiToStageData( + status: StageStatus, + stageInfo: StageInfo, + stageUiData: StageUIData, + includeDetails: Boolean): StageData = { + + val taskData = if (includeDetails) { + Some(stageUiData.taskData.map { case (k, v) => k -> convertTaskData(v) } ) + } else { + None + } + val executorSummary = if (includeDetails) { + Some(stageUiData.executorSummary.map { case (k, summary) => + k -> new ExecutorStageSummary( + taskTime = summary.taskTime, + failedTasks = summary.failedTasks, + succeededTasks = summary.succeededTasks, + inputBytes = summary.inputBytes, + outputBytes = summary.outputBytes, + shuffleRead = summary.shuffleRead, + shuffleWrite = summary.shuffleWrite, + memoryBytesSpilled = summary.memoryBytesSpilled, + diskBytesSpilled = summary.diskBytesSpilled + ) + }) + } else { + None + } + + val accumulableInfo = stageUiData.accumulables.values.map { convertAccumulableInfo }.toSeq + + new StageData( + status = status, + stageId = stageInfo.stageId, + attemptId = stageInfo.attemptId, + numActiveTasks = stageUiData.numActiveTasks, + numCompleteTasks = stageUiData.numCompleteTasks, + numFailedTasks = stageUiData.numFailedTasks, + executorRunTime = stageUiData.executorRunTime, + inputBytes = stageUiData.inputBytes, + inputRecords = stageUiData.inputRecords, + outputBytes = stageUiData.outputBytes, + outputRecords = stageUiData.outputRecords, + shuffleReadBytes = stageUiData.shuffleReadTotalBytes, + shuffleReadRecords = stageUiData.shuffleReadRecords, + shuffleWriteBytes = stageUiData.shuffleWriteBytes, + shuffleWriteRecords = stageUiData.shuffleWriteRecords, + memoryBytesSpilled = stageUiData.memoryBytesSpilled, + diskBytesSpilled = stageUiData.diskBytesSpilled, + schedulingPool = stageUiData.schedulingPool, + name = stageInfo.name, + details = stageInfo.details, + accumulatorUpdates = accumulableInfo, + tasks = taskData, + executorSummary = executorSummary + ) + } + + def stagesAndStatus(ui: SparkUI): Seq[(StageStatus, Seq[StageInfo])] = { + val listener = ui.jobProgressListener + listener.synchronized { + Seq( + StageStatus.ACTIVE -> listener.activeStages.values.toSeq, + StageStatus.COMPLETE -> listener.completedStages.reverse.toSeq, + StageStatus.FAILED -> listener.failedStages.reverse.toSeq, + StageStatus.PENDING -> listener.pendingStages.values.toSeq + ) + } + } + + def convertTaskData(uiData: TaskUIData): TaskData = { + new TaskData( + taskId = uiData.taskInfo.taskId, + index = uiData.taskInfo.index, + attempt = uiData.taskInfo.attempt, + launchTime = new Date(uiData.taskInfo.launchTime), + executorId = uiData.taskInfo.executorId, + host = uiData.taskInfo.host, + taskLocality = uiData.taskInfo.taskLocality.toString(), + speculative = uiData.taskInfo.speculative, + accumulatorUpdates = uiData.taskInfo.accumulables.map { convertAccumulableInfo }, + errorMessage = uiData.errorMessage, + taskMetrics = uiData.taskMetrics.map { convertUiTaskMetrics } + ) + } + + def taskMetricDistributions( + allTaskData: Iterable[TaskUIData], + quantiles: Array[Double]): TaskMetricDistributions = { + + val rawMetrics = allTaskData.flatMap{_.taskMetrics}.toSeq + + def metricQuantiles(f: InternalTaskMetrics => Double): IndexedSeq[Double] = + Distribution(rawMetrics.map { d => f(d) }).get.getQuantiles(quantiles) + + // We need to do a lot of similar munging to nested metrics here. For each one, + // we want (a) extract the values for nested metrics (b) make a distribution for each metric + // (c) shove the distribution into the right field in our return type and (d) only return + // a result if the option is defined for any of the tasks. MetricHelper is a little util + // to make it a little easier to deal w/ all of the nested options. Mostly it lets us just + // implement one "build" method, which just builds the quantiles for each field. + + val inputMetrics: Option[InputMetricDistributions] = + new MetricHelper[InternalInputMetrics, InputMetricDistributions](rawMetrics, quantiles) { + def getSubmetrics(raw: InternalTaskMetrics): Option[InternalInputMetrics] = { + raw.inputMetrics + } + + def build: InputMetricDistributions = new InputMetricDistributions( + bytesRead = submetricQuantiles(_.bytesRead), + recordsRead = submetricQuantiles(_.recordsRead) + ) + }.metricOption + + val outputMetrics: Option[OutputMetricDistributions] = + new MetricHelper[InternalOutputMetrics, OutputMetricDistributions](rawMetrics, quantiles) { + def getSubmetrics(raw:InternalTaskMetrics): Option[InternalOutputMetrics] = { + raw.outputMetrics + } + def build: OutputMetricDistributions = new OutputMetricDistributions( + bytesWritten = submetricQuantiles(_.bytesWritten), + recordsWritten = submetricQuantiles(_.recordsWritten) + ) + }.metricOption + + val shuffleReadMetrics: Option[ShuffleReadMetricDistributions] = + new MetricHelper[InternalShuffleReadMetrics, ShuffleReadMetricDistributions](rawMetrics, + quantiles) { + def getSubmetrics(raw: InternalTaskMetrics): Option[InternalShuffleReadMetrics] = { + raw.shuffleReadMetrics + } + def build: ShuffleReadMetricDistributions = new ShuffleReadMetricDistributions( + readBytes = submetricQuantiles(_.totalBytesRead), + readRecords = submetricQuantiles(_.recordsRead), + remoteBytesRead = submetricQuantiles(_.remoteBytesRead), + remoteBlocksFetched = submetricQuantiles(_.remoteBlocksFetched), + localBlocksFetched = submetricQuantiles(_.localBlocksFetched), + totalBlocksFetched = submetricQuantiles(_.totalBlocksFetched), + fetchWaitTime = submetricQuantiles(_.fetchWaitTime) + ) + }.metricOption + + val shuffleWriteMetrics: Option[ShuffleWriteMetricDistributions] = + new MetricHelper[InternalShuffleWriteMetrics, ShuffleWriteMetricDistributions](rawMetrics, + quantiles) { + def getSubmetrics(raw: InternalTaskMetrics): Option[InternalShuffleWriteMetrics] = { + raw.shuffleWriteMetrics + } + def build: ShuffleWriteMetricDistributions = new ShuffleWriteMetricDistributions( + writeBytes = submetricQuantiles(_.shuffleBytesWritten), + writeRecords = submetricQuantiles(_.shuffleRecordsWritten), + writeTime = submetricQuantiles(_.shuffleWriteTime) + ) + }.metricOption + + new TaskMetricDistributions( + quantiles = quantiles, + executorDeserializeTime = metricQuantiles(_.executorDeserializeTime), + executorRunTime = metricQuantiles(_.executorRunTime), + resultSize = metricQuantiles(_.resultSize), + jvmGcTime = metricQuantiles(_.jvmGCTime), + resultSerializationTime = metricQuantiles(_.resultSerializationTime), + memoryBytesSpilled = metricQuantiles(_.memoryBytesSpilled), + diskBytesSpilled = metricQuantiles(_.diskBytesSpilled), + inputMetrics = inputMetrics, + outputMetrics = outputMetrics, + shuffleReadMetrics = shuffleReadMetrics, + shuffleWriteMetrics = shuffleWriteMetrics + ) + } + + def convertAccumulableInfo(acc: InternalAccumulableInfo): AccumulableInfo = { + new AccumulableInfo(acc.id, acc.name, acc.update, acc.value) + } + + def convertUiTaskMetrics(internal: InternalTaskMetrics): TaskMetrics = { + new TaskMetrics( + executorDeserializeTime = internal.executorDeserializeTime, + executorRunTime = internal.executorRunTime, + resultSize = internal.resultSize, + jvmGcTime = internal.jvmGCTime, + resultSerializationTime = internal.resultSerializationTime, + memoryBytesSpilled = internal.memoryBytesSpilled, + diskBytesSpilled = internal.diskBytesSpilled, + inputMetrics = internal.inputMetrics.map { convertInputMetrics }, + outputMetrics = Option(internal.outputMetrics).flatten.map { convertOutputMetrics }, + shuffleReadMetrics = internal.shuffleReadMetrics.map { convertShuffleReadMetrics }, + shuffleWriteMetrics = internal.shuffleWriteMetrics.map { convertShuffleWriteMetrics } + ) + } + + def convertInputMetrics(internal: InternalInputMetrics): InputMetrics = { + new InputMetrics( + bytesRead = internal.bytesRead, + recordsRead = internal.recordsRead + ) + } + + def convertOutputMetrics(internal: InternalOutputMetrics): OutputMetrics = { + new OutputMetrics( + bytesWritten = internal.bytesWritten, + recordsWritten = internal.recordsWritten + ) + } + + def convertShuffleReadMetrics(internal: InternalShuffleReadMetrics): ShuffleReadMetrics = { + new ShuffleReadMetrics( + remoteBlocksFetched = internal.remoteBlocksFetched, + localBlocksFetched = internal.localBlocksFetched, + fetchWaitTime = internal.fetchWaitTime, + remoteBytesRead = internal.remoteBytesRead, + totalBlocksFetched = internal.totalBlocksFetched, + recordsRead = internal.recordsRead + ) + } + + def convertShuffleWriteMetrics(internal: InternalShuffleWriteMetrics): ShuffleWriteMetrics = { + new ShuffleWriteMetrics( + bytesWritten = internal.shuffleBytesWritten, + writeTime = internal.shuffleWriteTime, + recordsWritten = internal.shuffleRecordsWritten + ) + } +} + +/** + * Helper for getting distributions from nested metric types. Many of the metrics we want are + * contained in options inside TaskMetrics (eg., ShuffleWriteMetrics). This makes it easy to handle + * the options (returning None if the metrics are all empty), and extract the quantiles for each + * metric. After creating an instance, call metricOption to get the result type. + */ +private[v1] abstract class MetricHelper[I,O]( + rawMetrics: Seq[InternalTaskMetrics], + quantiles: Array[Double]) { + + def getSubmetrics(raw: InternalTaskMetrics): Option[I] + + def build: O + + val data: Seq[I] = rawMetrics.flatMap(getSubmetrics) + + /** applies the given function to all input metrics, and returns the quantiles */ + def submetricQuantiles(f: I => Double): IndexedSeq[Double] = { + Distribution(data.map { d => f(d) }).get.getQuantiles(quantiles) + } + + def metricOption: Option[O] = { + if (data.isEmpty) { + None + } else { + Some(build) + } + } +} diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/ApplicationListResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/ApplicationListResource.scala new file mode 100644 index 0000000000000..17b521f3e1d41 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/status/api/v1/ApplicationListResource.scala @@ -0,0 +1,94 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.status.api.v1 + +import java.util.{Arrays, Date, List => JList} +import javax.ws.rs.{DefaultValue, GET, Produces, QueryParam} +import javax.ws.rs.core.MediaType + +import org.apache.spark.deploy.history.ApplicationHistoryInfo +import org.apache.spark.deploy.master.{ApplicationInfo => InternalApplicationInfo} + +@Produces(Array(MediaType.APPLICATION_JSON)) +private[v1] class ApplicationListResource(uiRoot: UIRoot) { + + @GET + def appList( + @QueryParam("status") status: JList[ApplicationStatus], + @DefaultValue("2010-01-01") @QueryParam("minDate") minDate: SimpleDateParam, + @DefaultValue("3000-01-01") @QueryParam("maxDate") maxDate: SimpleDateParam) + : Iterator[ApplicationInfo] = { + val allApps = uiRoot.getApplicationInfoList + val adjStatus = { + if (status.isEmpty) { + Arrays.asList(ApplicationStatus.values(): _*) + } else { + status + } + } + val includeCompleted = adjStatus.contains(ApplicationStatus.COMPLETED) + val includeRunning = adjStatus.contains(ApplicationStatus.RUNNING) + allApps.filter { app => + val anyRunning = app.attempts.exists(!_.completed) + // if any attempt is still running, we consider the app to also still be running + val statusOk = (!anyRunning && includeCompleted) || + (anyRunning && includeRunning) + // keep the app if *any* attempts fall in the right time window + val dateOk = app.attempts.exists { attempt => + attempt.startTime.getTime >= minDate.timestamp && + attempt.startTime.getTime <= maxDate.timestamp + } + statusOk && dateOk + } + } +} + +private[spark] object ApplicationsListResource { + def appHistoryInfoToPublicAppInfo(app: ApplicationHistoryInfo): ApplicationInfo = { + new ApplicationInfo( + id = app.id, + name = app.name, + attempts = app.attempts.map { internalAttemptInfo => + new ApplicationAttemptInfo( + attemptId = internalAttemptInfo.attemptId, + startTime = new Date(internalAttemptInfo.startTime), + endTime = new Date(internalAttemptInfo.endTime), + sparkUser = internalAttemptInfo.sparkUser, + completed = internalAttemptInfo.completed + ) + } + ) + } + + def convertApplicationInfo( + internal: InternalApplicationInfo, + completed: Boolean): ApplicationInfo = { + // standalone application info always has just one attempt + new ApplicationInfo( + id = internal.id, + name = internal.desc.name, + attempts = Seq(new ApplicationAttemptInfo( + attemptId = None, + startTime = new Date(internal.startTime), + endTime = new Date(internal.endTime), + sparkUser = internal.desc.user, + completed = completed + )) + ) + } + +} diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/ExecutorListResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/ExecutorListResource.scala new file mode 100644 index 0000000000000..8ad4656b4dada --- /dev/null +++ b/core/src/main/scala/org/apache/spark/status/api/v1/ExecutorListResource.scala @@ -0,0 +1,36 @@ +/* +* Licensed to the Apache Software Foundation (ASF) under one or more +* contributor license agreements. See the NOTICE file distributed with +* this work for additional information regarding copyright ownership. +* The ASF licenses this file to You under the Apache License, Version 2.0 +* (the "License"); you may not use this file except in compliance with +* the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. +*/ +package org.apache.spark.status.api.v1 + +import javax.ws.rs.{GET, PathParam, Produces} +import javax.ws.rs.core.MediaType + +import org.apache.spark.ui.SparkUI +import org.apache.spark.ui.exec.ExecutorsPage + +@Produces(Array(MediaType.APPLICATION_JSON)) +private[v1] class ExecutorListResource(ui: SparkUI) { + + @GET + def executorList(): Seq[ExecutorSummary] = { + val listener = ui.executorsListener + val storageStatusList = listener.storageStatusList + (0 until storageStatusList.size).map { statusId => + ExecutorsPage.getExecInfo(listener, statusId) + } + } +} diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/JacksonMessageWriter.scala b/core/src/main/scala/org/apache/spark/status/api/v1/JacksonMessageWriter.scala new file mode 100644 index 0000000000000..202a5191ad57d --- /dev/null +++ b/core/src/main/scala/org/apache/spark/status/api/v1/JacksonMessageWriter.scala @@ -0,0 +1,93 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.status.api.v1 + +import java.io.OutputStream +import java.lang.annotation.Annotation +import java.lang.reflect.Type +import java.text.SimpleDateFormat +import java.util.{Calendar, SimpleTimeZone} +import javax.ws.rs.Produces +import javax.ws.rs.core.{MediaType, MultivaluedMap} +import javax.ws.rs.ext.{MessageBodyWriter, Provider} + +import com.fasterxml.jackson.annotation.JsonInclude +import com.fasterxml.jackson.databind.{ObjectMapper, SerializationFeature} + +/** + * This class converts the POJO metric responses into json, using jackson. + * + * This doesn't follow the standard jersey-jackson plugin options, because we want to stick + * with an old version of jersey (since we have it from yarn anyway) and don't want to pull in lots + * of dependencies from a new plugin. + * + * Note that jersey automatically discovers this class based on its package and its annotations. + */ +@Provider +@Produces(Array(MediaType.APPLICATION_JSON)) +private[v1] class JacksonMessageWriter extends MessageBodyWriter[Object]{ + + val mapper = new ObjectMapper() { + override def writeValueAsString(t: Any): String = { + super.writeValueAsString(t) + } + } + mapper.registerModule(com.fasterxml.jackson.module.scala.DefaultScalaModule) + mapper.enable(SerializationFeature.INDENT_OUTPUT) + mapper.setSerializationInclusion(JsonInclude.Include.NON_NULL) + mapper.setDateFormat(JacksonMessageWriter.makeISODateFormat) + + override def isWriteable( + aClass: Class[_], + `type`: Type, + annotations: Array[Annotation], + mediaType: MediaType): Boolean = { + true + } + + override def writeTo( + t: Object, + aClass: Class[_], + `type`: Type, + annotations: Array[Annotation], + mediaType: MediaType, + multivaluedMap: MultivaluedMap[String, AnyRef], + outputStream: OutputStream): Unit = { + t match { + case ErrorWrapper(err) => outputStream.write(err.getBytes("utf-8")) + case _ => mapper.writeValue(outputStream, t) + } + } + + override def getSize( + t: Object, + aClass: Class[_], + `type`: Type, + annotations: Array[Annotation], + mediaType: MediaType): Long = { + -1L + } +} + +private[spark] object JacksonMessageWriter { + def makeISODateFormat: SimpleDateFormat = { + val iso8601 = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSS'GMT'") + val cal = Calendar.getInstance(new SimpleTimeZone(0, "GMT")) + iso8601.setCalendar(cal) + iso8601 + } +} diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/JsonRootResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/JsonRootResource.scala new file mode 100644 index 0000000000000..c3ec45f54681b --- /dev/null +++ b/core/src/main/scala/org/apache/spark/status/api/v1/JsonRootResource.scala @@ -0,0 +1,255 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.status.api.v1 + +import javax.servlet.ServletContext +import javax.ws.rs._ +import javax.ws.rs.core.{Context, Response} + +import com.sun.jersey.api.core.ResourceConfig +import com.sun.jersey.spi.container.servlet.ServletContainer +import org.eclipse.jetty.server.handler.ContextHandler +import org.eclipse.jetty.servlet.{ServletContextHandler, ServletHolder} + +import org.apache.spark.SecurityManager +import org.apache.spark.ui.SparkUI + +/** + * Main entry point for serving spark application metrics as json, using JAX-RS. + * + * Each resource should have endpoints that return **public** classes defined in api.scala. Mima + * binary compatibility checks ensure that we don't inadvertently make changes that break the api. + * The returned objects are automatically converted to json by jackson with JacksonMessageWriter. + * In addition, there are a number of tests in HistoryServerSuite that compare the json to "golden + * files". Any changes and additions should be reflected there as well -- see the notes in + * HistoryServerSuite. + */ +@Path("/v1") +private[v1] class JsonRootResource extends UIRootFromServletContext { + + @Path("applications") + def getApplicationList(): ApplicationListResource = { + new ApplicationListResource(uiRoot) + } + + @Path("applications/{appId}") + def getApplication(): OneApplicationResource = { + new OneApplicationResource(uiRoot) + } + + @Path("applications/{appId}/{attemptId}/jobs") + def getJobs( + @PathParam("appId") appId: String, + @PathParam("attemptId") attemptId: String): AllJobsResource = { + uiRoot.withSparkUI(appId, Some(attemptId)) { ui => + new AllJobsResource(ui) + } + } + + @Path("applications/{appId}/jobs") + def getJobs(@PathParam("appId") appId: String): AllJobsResource = { + uiRoot.withSparkUI(appId, None) { ui => + new AllJobsResource(ui) + } + } + + @Path("applications/{appId}/jobs/{jobId: \\d+}") + def getJob(@PathParam("appId") appId: String): OneJobResource = { + uiRoot.withSparkUI(appId, None) { ui => + new OneJobResource(ui) + } + } + + @Path("applications/{appId}/{attemptId}/jobs/{jobId: \\d+}") + def getJob( + @PathParam("appId") appId: String, + @PathParam("attemptId") attemptId: String): OneJobResource = { + uiRoot.withSparkUI(appId, Some(attemptId)) { ui => + new OneJobResource(ui) + } + } + + @Path("applications/{appId}/executors") + def getExecutors(@PathParam("appId") appId: String): ExecutorListResource = { + uiRoot.withSparkUI(appId, None) { ui => + new ExecutorListResource(ui) + } + } + + @Path("applications/{appId}/{attemptId}/executors") + def getExecutors( + @PathParam("appId") appId: String, + @PathParam("attemptId") attemptId: String): ExecutorListResource = { + uiRoot.withSparkUI(appId, Some(attemptId)) { ui => + new ExecutorListResource(ui) + } + } + + + @Path("applications/{appId}/stages") + def getStages(@PathParam("appId") appId: String): AllStagesResource= { + uiRoot.withSparkUI(appId, None) { ui => + new AllStagesResource(ui) + } + } + + @Path("applications/{appId}/{attemptId}/stages") + def getStages( + @PathParam("appId") appId: String, + @PathParam("attemptId") attemptId: String): AllStagesResource= { + uiRoot.withSparkUI(appId, Some(attemptId)) { ui => + new AllStagesResource(ui) + } + } + + @Path("applications/{appId}/stages/{stageId: \\d+}") + def getStage(@PathParam("appId") appId: String): OneStageResource= { + uiRoot.withSparkUI(appId, None) { ui => + new OneStageResource(ui) + } + } + + @Path("applications/{appId}/{attemptId}/stages/{stageId: \\d+}") + def getStage( + @PathParam("appId") appId: String, + @PathParam("attemptId") attemptId: String): OneStageResource = { + uiRoot.withSparkUI(appId, Some(attemptId)) { ui => + new OneStageResource(ui) + } + } + + @Path("applications/{appId}/storage/rdd") + def getRdds(@PathParam("appId") appId: String): AllRDDResource = { + uiRoot.withSparkUI(appId, None) { ui => + new AllRDDResource(ui) + } + } + + @Path("applications/{appId}/{attemptId}/storage/rdd") + def getRdds( + @PathParam("appId") appId: String, + @PathParam("attemptId") attemptId: String): AllRDDResource = { + uiRoot.withSparkUI(appId, Some(attemptId)) { ui => + new AllRDDResource(ui) + } + } + + @Path("applications/{appId}/storage/rdd/{rddId: \\d+}") + def getRdd(@PathParam("appId") appId: String): OneRDDResource = { + uiRoot.withSparkUI(appId, None) { ui => + new OneRDDResource(ui) + } + } + + @Path("applications/{appId}/{attemptId}/storage/rdd/{rddId: \\d+}") + def getRdd( + @PathParam("appId") appId: String, + @PathParam("attemptId") attemptId: String): OneRDDResource = { + uiRoot.withSparkUI(appId, Some(attemptId)) { ui => + new OneRDDResource(ui) + } + } + +} + +private[spark] object JsonRootResource { + + def getJsonServlet(uiRoot: UIRoot): ServletContextHandler = { + val jerseyContext = new ServletContextHandler(ServletContextHandler.NO_SESSIONS) + jerseyContext.setContextPath("/json") + val holder:ServletHolder = new ServletHolder(classOf[ServletContainer]) + holder.setInitParameter("com.sun.jersey.config.property.resourceConfigClass", + "com.sun.jersey.api.core.PackagesResourceConfig") + holder.setInitParameter("com.sun.jersey.config.property.packages", + "org.apache.spark.status.api.v1") + holder.setInitParameter(ResourceConfig.PROPERTY_CONTAINER_REQUEST_FILTERS, + classOf[SecurityFilter].getCanonicalName) + UIRootFromServletContext.setUiRoot(jerseyContext, uiRoot) + jerseyContext.addServlet(holder, "/*") + jerseyContext + } +} + +/** + * This trait is shared by the all the root containers for application UI information -- + * the HistoryServer, the Master UI, and the application UI. This provides the common + * interface needed for them all to expose application info as json. + */ +private[spark] trait UIRoot { + def getSparkUI(appKey: String): Option[SparkUI] + def getApplicationInfoList: Iterator[ApplicationInfo] + + /** + * Get the spark UI with the given appID, and apply a function + * to it. If there is no such app, throw an appropriate exception + */ + def withSparkUI[T](appId: String, attemptId: Option[String])(f: SparkUI => T): T = { + val appKey = attemptId.map(appId + "/" + _).getOrElse(appId) + getSparkUI(appKey) match { + case Some(ui) => + f(ui) + case None => throw new NotFoundException("no such app: " + appId) + } + } + def securityManager: SecurityManager +} + +private[v1] object UIRootFromServletContext { + + private val attribute = getClass.getCanonicalName + + def setUiRoot(contextHandler: ContextHandler, uiRoot: UIRoot): Unit = { + contextHandler.setAttribute(attribute, uiRoot) + } + + def getUiRoot(context: ServletContext): UIRoot = { + context.getAttribute(attribute).asInstanceOf[UIRoot] + } +} + +private[v1] trait UIRootFromServletContext { + @Context + var servletContext: ServletContext = _ + + def uiRoot: UIRoot = UIRootFromServletContext.getUiRoot(servletContext) +} + +private[v1] class NotFoundException(msg: String) extends WebApplicationException( + new NoSuchElementException(msg), + Response + .status(Response.Status.NOT_FOUND) + .entity(ErrorWrapper(msg)) + .build() +) + +private[v1] class BadParameterException(msg: String) extends WebApplicationException( + new IllegalArgumentException(msg), + Response + .status(Response.Status.BAD_REQUEST) + .entity(ErrorWrapper(msg)) + .build() +) { + def this(param: String, exp: String, actual: String) = { + this(raw"""Bad value for parameter "$param". Expected a $exp, got "$actual"""") + } +} + +/** + * Signal to JacksonMessageWriter to not convert the message into json (which would result in an + * extra set of quotes). + */ +private[v1] case class ErrorWrapper(s: String) diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/OneApplicationResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/OneApplicationResource.scala new file mode 100644 index 0000000000000..b5ef72649e295 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/status/api/v1/OneApplicationResource.scala @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.status.api.v1 + +import javax.ws.rs.core.MediaType +import javax.ws.rs.{Produces, PathParam, GET} + +@Produces(Array(MediaType.APPLICATION_JSON)) +private[v1] class OneApplicationResource(uiRoot: UIRoot) { + + @GET + def getApp(@PathParam("appId") appId: String): ApplicationInfo = { + val apps = uiRoot.getApplicationInfoList.find { _.id == appId } + apps.getOrElse(throw new NotFoundException("unknown app: " + appId)) + } + +} diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/OneJobResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/OneJobResource.scala new file mode 100644 index 0000000000000..6d8a60d480aed --- /dev/null +++ b/core/src/main/scala/org/apache/spark/status/api/v1/OneJobResource.scala @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.status.api.v1 + +import javax.ws.rs.{PathParam, GET, Produces} +import javax.ws.rs.core.MediaType + +import org.apache.spark.JobExecutionStatus +import org.apache.spark.ui.SparkUI +import org.apache.spark.ui.jobs.UIData.JobUIData + +@Produces(Array(MediaType.APPLICATION_JSON)) +private[v1] class OneJobResource(ui: SparkUI) { + + @GET + def oneJob(@PathParam("jobId") jobId: Int): JobData = { + val statusToJobs: Seq[(JobExecutionStatus, Seq[JobUIData])] = + AllJobsResource.getStatusToJobs(ui) + val jobOpt = statusToJobs.map {_._2} .flatten.find { jobInfo => jobInfo.jobId == jobId} + jobOpt.map { job => + AllJobsResource.convertJobData(job, ui.jobProgressListener, false) + }.getOrElse { + throw new NotFoundException("unknown job: " + jobId) + } + } + +} diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/OneRDDResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/OneRDDResource.scala new file mode 100644 index 0000000000000..07b224fac4786 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/status/api/v1/OneRDDResource.scala @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.status.api.v1 + +import javax.ws.rs.{PathParam, GET, Produces} +import javax.ws.rs.core.MediaType + +import org.apache.spark.ui.SparkUI + +@Produces(Array(MediaType.APPLICATION_JSON)) +private[v1] class OneRDDResource(ui: SparkUI) { + + @GET + def rddData(@PathParam("rddId") rddId: Int): RDDStorageInfo = { + AllRDDResource.getRDDStorageInfo(rddId, ui.storageListener, true).getOrElse( + throw new NotFoundException(s"no rdd found w/ id $rddId") + ) + } + +} diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/OneStageResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/OneStageResource.scala new file mode 100644 index 0000000000000..fd24aea63a8a1 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/status/api/v1/OneStageResource.scala @@ -0,0 +1,150 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.status.api.v1 + +import javax.ws.rs._ +import javax.ws.rs.core.MediaType + +import org.apache.spark.SparkException +import org.apache.spark.scheduler.StageInfo +import org.apache.spark.status.api.v1.StageStatus._ +import org.apache.spark.status.api.v1.TaskSorting._ +import org.apache.spark.ui.SparkUI +import org.apache.spark.ui.jobs.JobProgressListener +import org.apache.spark.ui.jobs.UIData.StageUIData + +@Produces(Array(MediaType.APPLICATION_JSON)) +private[v1] class OneStageResource(ui: SparkUI) { + + @GET + @Path("") + def stageData(@PathParam("stageId") stageId: Int): Seq[StageData] = { + withStage(stageId){ stageAttempts => + stageAttempts.map { stage => + AllStagesResource.stageUiToStageData(stage.status, stage.info, stage.ui, + includeDetails = true) + } + } + } + + @GET + @Path("/{stageAttemptId: \\d+}") + def oneAttemptData( + @PathParam("stageId") stageId: Int, + @PathParam("stageAttemptId") stageAttemptId: Int): StageData = { + withStageAttempt(stageId, stageAttemptId) { stage => + AllStagesResource.stageUiToStageData(stage.status, stage.info, stage.ui, + includeDetails = true) + } + } + + @GET + @Path("/{stageAttemptId: \\d+}/taskSummary") + def taskSummary( + @PathParam("stageId") stageId: Int, + @PathParam("stageAttemptId") stageAttemptId: Int, + @DefaultValue("0.05,0.25,0.5,0.75,0.95") @QueryParam("quantiles") quantileString: String) + : TaskMetricDistributions = { + withStageAttempt(stageId, stageAttemptId) { stage => + val quantiles = quantileString.split(",").map { s => + try { + s.toDouble + } catch { + case nfe: NumberFormatException => + throw new BadParameterException("quantiles", "double", s) + } + } + AllStagesResource.taskMetricDistributions(stage.ui.taskData.values, quantiles) + } + } + + @GET + @Path("/{stageAttemptId: \\d+}/taskList") + def taskList( + @PathParam("stageId") stageId: Int, + @PathParam("stageAttemptId") stageAttemptId: Int, + @DefaultValue("0") @QueryParam("offset") offset: Int, + @DefaultValue("20") @QueryParam("length") length: Int, + @DefaultValue("ID") @QueryParam("sortBy") sortBy: TaskSorting): Seq[TaskData] = { + withStageAttempt(stageId, stageAttemptId) { stage => + val tasks = stage.ui.taskData.values.map{AllStagesResource.convertTaskData}.toIndexedSeq + .sorted(OneStageResource.ordering(sortBy)) + tasks.slice(offset, offset + length) + } + } + + private case class StageStatusInfoUi(status: StageStatus, info: StageInfo, ui: StageUIData) + + private def withStage[T](stageId: Int)(f: Seq[StageStatusInfoUi] => T): T = { + val stageAttempts = findStageStatusUIData(ui.jobProgressListener, stageId) + if (stageAttempts.isEmpty) { + throw new NotFoundException("unknown stage: " + stageId) + } else { + f(stageAttempts) + } + } + + private def findStageStatusUIData( + listener: JobProgressListener, + stageId: Int): Seq[StageStatusInfoUi] = { + listener.synchronized { + def getStatusInfoUi(status: StageStatus, infos: Seq[StageInfo]): Seq[StageStatusInfoUi] = { + infos.filter { _.stageId == stageId }.map { info => + val ui = listener.stageIdToData.getOrElse((info.stageId, info.attemptId), + // this is an internal error -- we should always have uiData + throw new SparkException( + s"no stage ui data found for stage: ${info.stageId}:${info.attemptId}") + ) + StageStatusInfoUi(status, info, ui) + } + } + getStatusInfoUi(ACTIVE, listener.activeStages.values.toSeq) ++ + getStatusInfoUi(COMPLETE, listener.completedStages) ++ + getStatusInfoUi(FAILED, listener.failedStages) ++ + getStatusInfoUi(PENDING, listener.pendingStages.values.toSeq) + } + } + + private def withStageAttempt[T]( + stageId: Int, + stageAttemptId: Int) + (f: StageStatusInfoUi => T): T = { + withStage(stageId) { attempts => + val oneAttempt = attempts.find { stage => stage.info.attemptId == stageAttemptId } + oneAttempt match { + case Some(stage) => + f(stage) + case None => + val stageAttempts = attempts.map { _.info.attemptId } + throw new NotFoundException(s"unknown attempt for stage $stageId. " + + s"Found attempts: ${stageAttempts.mkString("[", ",", "]")}") + } + } + } +} + +object OneStageResource { + def ordering(taskSorting: TaskSorting): Ordering[TaskData] = { + val extractor: (TaskData => Long) = td => + taskSorting match { + case ID => td.taskId + case INCREASING_RUNTIME => td.taskMetrics.map{_.executorRunTime}.getOrElse(-1L) + case DECREASING_RUNTIME => -td.taskMetrics.map{_.executorRunTime}.getOrElse(-1L) + } + Ordering.by(extractor) + } +} diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/SecurityFilter.scala b/core/src/main/scala/org/apache/spark/status/api/v1/SecurityFilter.scala new file mode 100644 index 0000000000000..95fbd96ade5ab --- /dev/null +++ b/core/src/main/scala/org/apache/spark/status/api/v1/SecurityFilter.scala @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.status.api.v1 + +import javax.ws.rs.WebApplicationException +import javax.ws.rs.core.Response + +import com.sun.jersey.spi.container.{ContainerRequest, ContainerRequestFilter} + +private[v1] class SecurityFilter extends ContainerRequestFilter with UIRootFromServletContext { + def filter(req: ContainerRequest): ContainerRequest = { + val user = Option(req.getUserPrincipal).map { _.getName }.orNull + if (uiRoot.securityManager.checkUIViewPermissions(user)) { + req + } else { + throw new WebApplicationException( + Response + .status(Response.Status.FORBIDDEN) + .entity(raw"""user "$user"is not authorized""") + .build() + ) + } + } +} diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/SimpleDateParam.scala b/core/src/main/scala/org/apache/spark/status/api/v1/SimpleDateParam.scala new file mode 100644 index 0000000000000..cee29786c3019 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/status/api/v1/SimpleDateParam.scala @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.status.api.v1 + +import java.text.SimpleDateFormat +import java.util.TimeZone +import javax.ws.rs.WebApplicationException +import javax.ws.rs.core.Response +import javax.ws.rs.core.Response.Status + +import scala.util.Try + +private[v1] class SimpleDateParam(val originalValue: String) { + val timestamp: Long = { + SimpleDateParam.formats.collectFirst { + case fmt if Try(fmt.parse(originalValue)).isSuccess => + fmt.parse(originalValue).getTime() + }.getOrElse( + throw new WebApplicationException( + Response + .status(Status.BAD_REQUEST) + .entity("Couldn't parse date: " + originalValue) + .build() + ) + ) + } +} + +private[v1] object SimpleDateParam { + + val formats: Seq[SimpleDateFormat] = { + + val gmtDay = new SimpleDateFormat("yyyy-MM-dd") + gmtDay.setTimeZone(TimeZone.getTimeZone("GMT")) + + Seq( + new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSz"), + gmtDay + ) + } +} 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 new file mode 100644 index 0000000000000..ef3c8570d8186 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala @@ -0,0 +1,228 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.status.api.v1 + +import java.util.Date + +import scala.collection.Map + +import org.apache.spark.JobExecutionStatus + +class ApplicationInfo private[spark]( + val id: String, + val name: String, + val attempts: Seq[ApplicationAttemptInfo]) + +class ApplicationAttemptInfo private[spark]( + val attemptId: Option[String], + val startTime: Date, + val endTime: Date, + val sparkUser: String, + val completed: Boolean = false) + +class ExecutorStageSummary private[spark]( + val taskTime : Long, + val failedTasks : Int, + val succeededTasks : Int, + val inputBytes : Long, + val outputBytes : Long, + val shuffleRead : Long, + val shuffleWrite : Long, + val memoryBytesSpilled : Long, + val diskBytesSpilled : Long) + +class ExecutorSummary private[spark]( + val id: String, + val hostPort: String, + val rddBlocks: Int, + val memoryUsed: Long, + val diskUsed: Long, + val activeTasks: Int, + val failedTasks: Int, + val completedTasks: Int, + val totalTasks: Int, + val totalDuration: Long, + val totalInputBytes: Long, + val totalShuffleRead: Long, + val totalShuffleWrite: Long, + val maxMemory: Long, + val executorLogs: Map[String, String]) + +class JobData private[spark]( + val jobId: Int, + val name: String, + val description: Option[String], + val submissionTime: Option[Date], + val completionTime: Option[Date], + val stageIds: Seq[Int], + val jobGroup: Option[String], + val status: JobExecutionStatus, + val numTasks: Int, + val numActiveTasks: Int, + val numCompletedTasks: Int, + val numSkippedTasks: Int, + val numFailedTasks: Int, + val numActiveStages: Int, + val numCompletedStages: Int, + val numSkippedStages: Int, + val numFailedStages: Int) + +// Q: should Tachyon size go in here as well? currently the UI only shows it on the overall storage +// page ... does anybody pay attention to it? +class RDDStorageInfo private[spark]( + val id: Int, + val name: String, + val numPartitions: Int, + val numCachedPartitions: Int, + val storageLevel: String, + val memoryUsed: Long, + val diskUsed: Long, + val dataDistribution: Option[Seq[RDDDataDistribution]], + val partitions: Option[Seq[RDDPartitionInfo]]) + +class RDDDataDistribution private[spark]( + val address: String, + val memoryUsed: Long, + val memoryRemaining: Long, + val diskUsed: Long) + +class RDDPartitionInfo private[spark]( + val blockName: String, + val storageLevel: String, + val memoryUsed: Long, + val diskUsed: Long, + val executors: Seq[String]) + +class StageData private[spark]( + val status: StageStatus, + val stageId: Int, + val attemptId: Int, + val numActiveTasks: Int , + val numCompleteTasks: Int, + val numFailedTasks: Int, + + val executorRunTime: Long, + + val inputBytes: Long, + val inputRecords: Long, + val outputBytes: Long, + val outputRecords: Long, + val shuffleReadBytes: Long, + val shuffleReadRecords: Long, + val shuffleWriteBytes: Long, + val shuffleWriteRecords: Long, + val memoryBytesSpilled: Long, + val diskBytesSpilled: Long, + + val name: String, + val details: String, + val schedulingPool: String, + + val accumulatorUpdates: Seq[AccumulableInfo], + val tasks: Option[Map[Long, TaskData]], + val executorSummary:Option[Map[String,ExecutorStageSummary]]) + +class TaskData private[spark]( + val taskId: Long, + val index: Int, + val attempt: Int, + val launchTime: Date, + val executorId: String, + val host: String, + val taskLocality: String, + val speculative: Boolean, + val accumulatorUpdates: Seq[AccumulableInfo], + val errorMessage: Option[String] = None, + val taskMetrics: Option[TaskMetrics] = None) + +class TaskMetrics private[spark]( + val executorDeserializeTime: Long, + val executorRunTime: Long, + val resultSize: Long, + val jvmGcTime: Long, + val resultSerializationTime: Long, + val memoryBytesSpilled: Long, + val diskBytesSpilled: Long, + val inputMetrics: Option[InputMetrics], + val outputMetrics: Option[OutputMetrics], + val shuffleReadMetrics: Option[ShuffleReadMetrics], + val shuffleWriteMetrics: Option[ShuffleWriteMetrics]) + +class InputMetrics private[spark]( + val bytesRead: Long, + val recordsRead: Long) + +class OutputMetrics private[spark]( + val bytesWritten: Long, + val recordsWritten: Long) + +class ShuffleReadMetrics private[spark]( + val remoteBlocksFetched: Int, + val localBlocksFetched: Int, + val fetchWaitTime: Long, + val remoteBytesRead: Long, + val totalBlocksFetched: Int, + val recordsRead: Long) + +class ShuffleWriteMetrics private[spark]( + val bytesWritten: Long, + val writeTime: Long, + val recordsWritten: Long) + +class TaskMetricDistributions private[spark]( + val quantiles: IndexedSeq[Double], + + val executorDeserializeTime: IndexedSeq[Double], + val executorRunTime: IndexedSeq[Double], + val resultSize: IndexedSeq[Double], + val jvmGcTime: IndexedSeq[Double], + val resultSerializationTime: IndexedSeq[Double], + val memoryBytesSpilled: IndexedSeq[Double], + val diskBytesSpilled: IndexedSeq[Double], + + val inputMetrics: Option[InputMetricDistributions], + val outputMetrics: Option[OutputMetricDistributions], + val shuffleReadMetrics: Option[ShuffleReadMetricDistributions], + val shuffleWriteMetrics: Option[ShuffleWriteMetricDistributions]) + +class InputMetricDistributions private[spark]( + val bytesRead: IndexedSeq[Double], + val recordsRead: IndexedSeq[Double]) + +class OutputMetricDistributions private[spark]( + val bytesWritten: IndexedSeq[Double], + val recordsWritten: IndexedSeq[Double]) + +class ShuffleReadMetricDistributions private[spark]( + val readBytes: IndexedSeq[Double], + val readRecords: IndexedSeq[Double], + val remoteBlocksFetched: IndexedSeq[Double], + val localBlocksFetched: IndexedSeq[Double], + val fetchWaitTime: IndexedSeq[Double], + val remoteBytesRead: IndexedSeq[Double], + val totalBlocksFetched: IndexedSeq[Double]) + +class ShuffleWriteMetricDistributions private[spark]( + val writeBytes: IndexedSeq[Double], + val writeRecords: IndexedSeq[Double], + val writeTime: IndexedSeq[Double]) + +class AccumulableInfo private[spark]( + val id: Long, + val name: String, + val update: Option[String], + val value: String) diff --git a/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala b/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala index 7d75929b96f75..ec711480ebf30 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala @@ -25,13 +25,17 @@ import org.apache.spark.scheduler._ /** * :: DeveloperApi :: * A SparkListener that maintains executor storage status. + * + * This class is thread-safe (unlike JobProgressListener) */ @DeveloperApi class StorageStatusListener extends SparkListener { // This maintains only blocks that are cached (i.e. storage level is not StorageLevel.NONE) private[storage] val executorIdToStorageStatus = mutable.Map[String, StorageStatus]() - def storageStatusList: Seq[StorageStatus] = executorIdToStorageStatus.values.toSeq + def storageStatusList: Seq[StorageStatus] = synchronized { + executorIdToStorageStatus.values.toSeq + } /** Update storage status list to reflect updated block statuses */ private def updateStorageStatus(execId: String, updatedBlocks: Seq[(BlockId, BlockStatus)]) { diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index a5271f0574e6c..bfe4a180e8a6f 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -17,6 +17,9 @@ package org.apache.spark.ui +import java.util.Date + +import org.apache.spark.status.api.v1.{ApplicationAttemptInfo, ApplicationInfo, JsonRootResource, UIRoot} import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext} import org.apache.spark.scheduler._ import org.apache.spark.storage.StorageStatusListener @@ -33,7 +36,7 @@ import org.apache.spark.ui.scope.RDDOperationGraphListener private[spark] class SparkUI private ( val sc: Option[SparkContext], val conf: SparkConf, - val securityManager: SecurityManager, + securityManager: SecurityManager, val environmentListener: EnvironmentListener, val storageStatusListener: StorageStatusListener, val executorsListener: ExecutorsListener, @@ -41,22 +44,27 @@ private[spark] class SparkUI private ( val storageListener: StorageListener, val operationGraphListener: RDDOperationGraphListener, var appName: String, - val basePath: String) + val basePath: String, + val startTime: Long) extends WebUI(securityManager, SparkUI.getUIPort(conf), conf, basePath, "SparkUI") - with Logging { + with Logging + with UIRoot { val killEnabled = sc.map(_.conf.getBoolean("spark.ui.killEnabled", true)).getOrElse(false) + + val stagesTab = new StagesTab(this) + /** Initialize all components of the server. */ def initialize() { attachTab(new JobsTab(this)) - val stagesTab = new StagesTab(this) attachTab(stagesTab) attachTab(new StorageTab(this)) attachTab(new EnvironmentTab(this)) attachTab(new ExecutorsTab(this)) attachHandler(createStaticHandler(SparkUI.STATIC_RESOURCE_DIR, "/static")) attachHandler(createRedirectHandler("/", "/jobs", basePath = basePath)) + attachHandler(JsonRootResource.getJsonServlet(this)) // This should be POST only, but, the YARN AM proxy won't proxy POSTs attachHandler(createRedirectHandler( "/stages/stage/kill", "/stages", stagesTab.handleKillRequest, @@ -83,6 +91,24 @@ private[spark] class SparkUI private ( private[spark] def appUIHostPort = publicHostName + ":" + boundPort private[spark] def appUIAddress = s"http://$appUIHostPort" + + def getSparkUI(appId: String): Option[SparkUI] = { + if (appId == appName) Some(this) else None + } + + def getApplicationInfoList: Iterator[ApplicationInfo] = { + Iterator(new ApplicationInfo( + id = appName, + name = appName, + attempts = Seq(new ApplicationAttemptInfo( + attemptId = None, + startTime = new Date(startTime), + endTime = new Date(-1), + sparkUser = "", + completed = false + )) + )) + } } private[spark] abstract class SparkUITab(parent: SparkUI, prefix: String) @@ -109,9 +135,10 @@ private[spark] object SparkUI { listenerBus: SparkListenerBus, jobProgressListener: JobProgressListener, securityManager: SecurityManager, - appName: String): SparkUI = { + appName: String, + startTime: Long): SparkUI = { create(Some(sc), conf, listenerBus, securityManager, appName, - jobProgressListener = Some(jobProgressListener)) + jobProgressListener = Some(jobProgressListener), startTime = startTime) } def createHistoryUI( @@ -119,8 +146,9 @@ private[spark] object SparkUI { listenerBus: SparkListenerBus, securityManager: SecurityManager, appName: String, - basePath: String): SparkUI = { - create(None, conf, listenerBus, securityManager, appName, basePath) + basePath: String, + startTime: Long): SparkUI = { + create(None, conf, listenerBus, securityManager, appName, basePath, startTime = startTime) } /** @@ -137,7 +165,8 @@ private[spark] object SparkUI { securityManager: SecurityManager, appName: String, basePath: String = "", - jobProgressListener: Option[JobProgressListener] = None): SparkUI = { + jobProgressListener: Option[JobProgressListener] = None, + startTime: Long): SparkUI = { val _jobProgressListener: JobProgressListener = jobProgressListener.getOrElse { val listener = new JobProgressListener(conf) @@ -159,6 +188,6 @@ private[spark] object SparkUI { new SparkUI(sc, conf, securityManager, environmentListener, storageStatusListener, executorsListener, _jobProgressListener, storageListener, operationGraphListener, - appName, basePath) + appName, basePath, startTime) } } diff --git a/core/src/main/scala/org/apache/spark/ui/WebUI.scala b/core/src/main/scala/org/apache/spark/ui/WebUI.scala index f9860d1a5ce76..384f2ad26e281 100644 --- a/core/src/main/scala/org/apache/spark/ui/WebUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/WebUI.scala @@ -37,7 +37,7 @@ import org.apache.spark.{Logging, SecurityManager, SparkConf} * pages. The use of tabs is optional, however; a WebUI may choose to include pages directly. */ private[spark] abstract class WebUI( - securityManager: SecurityManager, + val securityManager: SecurityManager, port: Int, conf: SparkConf, basePath: String = "", @@ -77,15 +77,9 @@ private[spark] abstract class WebUI( val pagePath = "/" + page.prefix val renderHandler = createServletHandler(pagePath, (request: HttpServletRequest) => page.render(request), securityManager, basePath) - val renderJsonHandler = createServletHandler(pagePath.stripSuffix("/") + "/json", - (request: HttpServletRequest) => page.renderJson(request), securityManager, basePath) attachHandler(renderHandler) - attachHandler(renderJsonHandler) pageToHandlers.getOrElseUpdate(page, ArrayBuffer[ServletContextHandler]()) .append(renderHandler) - pageToHandlers.getOrElseUpdate(page, ArrayBuffer[ServletContextHandler]()) - .append(renderJsonHandler) - } /** Attach a handler to this UI. */ diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala index 956608d7c0cbe..b247e4cdc3bd4 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala @@ -22,11 +22,11 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node +import org.apache.spark.status.api.v1.ExecutorSummary import org.apache.spark.ui.{ToolTips, UIUtils, WebUIPage} import org.apache.spark.util.Utils -/** Summary information about an executor to display in the UI. */ -// Needs to be private[ui] because of a false positive MiMa failure. +// This isn't even used anymore -- but we need to keep it b/c of a MiMa false positive private[ui] case class ExecutorSummaryInfo( id: String, hostPort: String, @@ -44,6 +44,7 @@ private[ui] case class ExecutorSummaryInfo( maxMemory: Long, executorLogs: Map[String, String]) + private[ui] class ExecutorsPage( parent: ExecutorsTab, threadDumpEnabled: Boolean) @@ -55,7 +56,8 @@ private[ui] class ExecutorsPage( val maxMem = storageStatusList.map(_.maxMem).sum val memUsed = storageStatusList.map(_.memUsed).sum val diskUsed = storageStatusList.map(_.diskUsed).sum - val execInfo = for (statusId <- 0 until storageStatusList.size) yield getExecInfo(statusId) + val execInfo = for (statusId <- 0 until storageStatusList.size) yield + ExecutorsPage.getExecInfo(listener, statusId) val execInfoSorted = execInfo.sortBy(_.id) val logsExist = execInfo.filter(_.executorLogs.nonEmpty).nonEmpty @@ -111,7 +113,7 @@ private[ui] class ExecutorsPage( } /** Render an HTML row representing an executor */ - private def execRow(info: ExecutorSummaryInfo, logsExist: Boolean): Seq[Node] = { + private def execRow(info: ExecutorSummary, logsExist: Boolean): Seq[Node] = { val maximumMemory = info.maxMemory val memoryUsed = info.memoryUsed val diskUsed = info.diskUsed @@ -170,8 +172,11 @@ private[ui] class ExecutorsPage( } +} + +private[spark] object ExecutorsPage { /** Represent an executor's info as a map given a storage status index */ - private def getExecInfo(statusId: Int): ExecutorSummaryInfo = { + def getExecInfo(listener: ExecutorsListener, statusId: Int): ExecutorSummary = { val status = listener.storageStatusList(statusId) val execId = status.blockManagerId.executorId val hostPort = status.blockManagerId.hostPort @@ -189,7 +194,7 @@ private[ui] class ExecutorsPage( val totalShuffleWrite = listener.executorToShuffleWrite.getOrElse(execId, 0L) val executorLogs = listener.executorToLogUrls.getOrElse(execId, Map.empty) - new ExecutorSummaryInfo( + new ExecutorSummary( execId, hostPort, rddBlocks, diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala index f6abf27db49dd..09323d1d80ad6 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala @@ -271,6 +271,12 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { val shouldShowCompletedJobs = completedJobs.nonEmpty val shouldShowFailedJobs = failedJobs.nonEmpty + val completedJobNumStr = if (completedJobs.size == listener.numCompletedJobs) { + s"${completedJobs.size}" + } else { + s"${listener.numCompletedJobs}, only showing ${completedJobs.size}" + } + val summary: NodeSeq =
        @@ -295,9 +301,9 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { } { if (shouldShowCompletedJobs) { -
      • +
      • Completed Jobs: - {completedJobs.size} + {completedJobNumStr}
      • } } @@ -305,7 +311,7 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { if (shouldShowFailedJobs) {
      • Failed Jobs: - {failedJobs.size} + {listener.numFailedJobs}
      • } } @@ -322,7 +328,7 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { activeJobsTable } if (shouldShowCompletedJobs) { - content ++=

        Completed Jobs ({completedJobs.size})

        ++ + content ++=

        Completed Jobs ({completedJobNumStr})

        ++ completedJobsTable } if (shouldShowFailedJobs) { diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala index 236bc8ea92879..a37f739ab9c66 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala @@ -64,6 +64,12 @@ private[ui] class AllStagesPage(parent: StagesTab) extends WebUIPage("") { val shouldShowCompletedStages = completedStages.nonEmpty val shouldShowFailedStages = failedStages.nonEmpty + val completedStageNumStr = if (numCompletedStages == completedStages.size) { + s"$numCompletedStages" + } else { + s"$numCompletedStages, only showing ${completedStages.size}" + } + val summary: NodeSeq =
          @@ -98,9 +104,9 @@ private[ui] class AllStagesPage(parent: StagesTab) extends WebUIPage("") { } { if (shouldShowCompletedStages) { -
        • +
        • Completed Stages: - {numCompletedStages} + {completedStageNumStr}
        • } } @@ -132,7 +138,7 @@ private[ui] class AllStagesPage(parent: StagesTab) extends WebUIPage("") { pendingStagesTable.toNodeSeq } if (shouldShowCompletedStages) { - content ++=

          Completed Stages ({numCompletedStages})

          ++ + content ++=

          Completed Stages ({completedStageNumStr})

          ++ completedStagesTable.toNodeSeq } if (shouldShowFailedStages) { diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala index 96cc3d78d0f15..7163217e1fed0 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala @@ -187,7 +187,7 @@ private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") { val jobDataOption = listener.jobIdToData.get(jobId) if (jobDataOption.isEmpty) { val content = -
          +

          No information to display for job {jobId}

          return UIUtils.headerSparkPage( diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index 8f9aa9fdec819..246e191d64776 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -74,6 +74,8 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { // JobProgressListener's retention limits. var numCompletedStages = 0 var numFailedStages = 0 + var numCompletedJobs = 0 + var numFailedJobs = 0 // Misc: val executorIdToBlockManagerId = HashMap[ExecutorId, BlockManagerId]() @@ -217,10 +219,12 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { completedJobs += jobData trimJobsIfNecessary(completedJobs) jobData.status = JobExecutionStatus.SUCCEEDED + numCompletedJobs += 1 case JobFailed(exception) => failedJobs += jobData trimJobsIfNecessary(failedJobs) jobData.status = JobExecutionStatus.FAILED + numFailedJobs += 1 } for (stageId <- jobData.stageIds) { stageIdToActiveJobIds.get(stageId).foreach { jobsUsingStage => diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala index d725b9d8565ac..f3e0b38523f32 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala @@ -21,7 +21,7 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node -import org.apache.spark.scheduler.{Schedulable, StageInfo} +import org.apache.spark.scheduler.StageInfo import org.apache.spark.ui.{WebUIPage, UIUtils} /** Page showing specific pool details */ diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index 6c4305873cbd9..b01fad8e453c8 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -25,11 +25,11 @@ import scala.xml.{Elem, Node, Unparsed} import org.apache.commons.lang3.StringEscapeUtils import org.apache.spark.executor.TaskMetrics +import org.apache.spark.scheduler.{AccumulableInfo, TaskInfo} import org.apache.spark.ui.{ToolTips, WebUIPage, UIUtils} import org.apache.spark.ui.jobs.UIData._ import org.apache.spark.ui.scope.RDDOperationGraph import org.apache.spark.util.{Utils, Distribution} -import org.apache.spark.scheduler.{AccumulableInfo, TaskInfo} /** Page showing statistics and task list for a given stage */ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { @@ -52,14 +52,22 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { val stageAttemptId = parameterAttempt.toInt val stageDataOption = progressListener.stageIdToData.get((stageId, stageAttemptId)) - if (stageDataOption.isEmpty || stageDataOption.get.taskData.isEmpty) { + val stageHeader = s"Details for Stage $stageId (Attempt $stageAttemptId)" + if (stageDataOption.isEmpty) { + val content = +
          +

          No information to display for Stage {stageId} (Attempt {stageAttemptId})

          +
          + return UIUtils.headerSparkPage(stageHeader, content, parent) + + } + if (stageDataOption.get.taskData.isEmpty) { val content =

          Summary Metrics

          No tasks have started yet

          Tasks

          No tasks have started yet
          - return UIUtils.headerSparkPage( - s"Details for Stage $stageId (Attempt $stageAttemptId)", content, parent) + return UIUtils.headerSparkPage(stageHeader, content, parent) } val stageData = stageDataOption.get @@ -458,8 +466,7 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { maybeAccumulableTable ++

          Tasks

          ++ taskTable - UIUtils.headerSparkPage( - "Details for Stage %d".format(stageId), content, parent, showVisualization = true) + UIUtils.headerSparkPage(stageHeader, content, parent, showVisualization = true) } } diff --git a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala index 199f731b92bcc..05f94a7507f4f 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala @@ -21,8 +21,8 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node -import org.apache.spark.storage.{BlockId, BlockStatus, StorageStatus, StorageUtils} -import org.apache.spark.ui.{WebUIPage, UIUtils} +import org.apache.spark.status.api.v1.{AllRDDResource, RDDDataDistribution, RDDPartitionInfo} +import org.apache.spark.ui.{UIUtils, WebUIPage} import org.apache.spark.util.Utils /** Page showing storage details for a given RDD */ @@ -32,28 +32,19 @@ private[ui] class RDDPage(parent: StorageTab) extends WebUIPage("rdd") { def render(request: HttpServletRequest): Seq[Node] = { val parameterId = request.getParameter("id") require(parameterId != null && parameterId.nonEmpty, "Missing id parameter") - val rddId = parameterId.toInt - val storageStatusList = listener.storageStatusList - val rddInfo = listener.rddInfoList.find(_.id == rddId).getOrElse { - // Rather than crashing, render an "RDD Not Found" page - return UIUtils.headerSparkPage("RDD Not Found", Seq[Node](), parent) - } + val rddStorageInfo = AllRDDResource.getRDDStorageInfo(rddId, listener,includeDetails = true) + .getOrElse { + // Rather than crashing, render an "RDD Not Found" page + return UIUtils.headerSparkPage("RDD Not Found", Seq[Node](), parent) + } // Worker table - val workers = storageStatusList.map((rddId, _)) - val workerTable = UIUtils.listingTable(workerHeader, workerRow, workers, - id = Some("rdd-storage-by-worker-table")) + val workerTable = UIUtils.listingTable(workerHeader, workerRow, + rddStorageInfo.dataDistribution.get, id = Some("rdd-storage-by-worker-table")) // Block table - val blockLocations = StorageUtils.getRddBlockLocations(rddId, storageStatusList) - val blocks = storageStatusList - .flatMap(_.rddBlocksById(rddId)) - .sortWith(_._1.name < _._1.name) - .map { case (blockId, status) => - (blockId, status, blockLocations.get(blockId).getOrElse(Seq[String]("Unknown"))) - } - val blockTable = UIUtils.listingTable(blockHeader, blockRow, blocks, + val blockTable = UIUtils.listingTable(blockHeader, blockRow, rddStorageInfo.partitions.get, id = Some("rdd-storage-by-block-table")) val content = @@ -62,23 +53,23 @@ private[ui] class RDDPage(parent: StorageTab) extends WebUIPage("rdd") {
          • Storage Level: - {rddInfo.storageLevel.description} + {rddStorageInfo.storageLevel}
          • Cached Partitions: - {rddInfo.numCachedPartitions} + {rddStorageInfo.numCachedPartitions}
          • Total Partitions: - {rddInfo.numPartitions} + {rddStorageInfo.numPartitions}
          • Memory Size: - {Utils.bytesToString(rddInfo.memSize)} + {Utils.bytesToString(rddStorageInfo.memoryUsed)}
          • Disk Size: - {Utils.bytesToString(rddInfo.diskSize)} + {Utils.bytesToString(rddStorageInfo.diskUsed)}
          @@ -86,19 +77,19 @@ private[ui] class RDDPage(parent: StorageTab) extends WebUIPage("rdd") {
          -

          Data Distribution on {workers.size} Executors

          +

          Data Distribution on {rddStorageInfo.dataDistribution.size} Executors

          {workerTable}
          -

          {blocks.size} Partitions

          +

          {rddStorageInfo.partitions.size} Partitions

          {blockTable}
          ; - UIUtils.headerSparkPage("RDD Storage Info for " + rddInfo.name, content, parent) + UIUtils.headerSparkPage("RDD Storage Info for " + rddStorageInfo.name, content, parent) } /** Header fields for the worker table */ @@ -116,34 +107,32 @@ private[ui] class RDDPage(parent: StorageTab) extends WebUIPage("rdd") { "Executors") /** Render an HTML row representing a worker */ - private def workerRow(worker: (Int, StorageStatus)): Seq[Node] = { - val (rddId, status) = worker + private def workerRow(worker: RDDDataDistribution): Seq[Node] = {
      - + - + } /** Render an HTML row representing a block */ - private def blockRow(row: (BlockId, BlockStatus, Seq[String])): Seq[Node] = { - val (id, block, locations) = row + private def blockRow(row: RDDPartitionInfo): Seq[Node] = { - + - - } diff --git a/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala b/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala index 59dc6b547c1d8..07db783c572cf 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala @@ -22,7 +22,7 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node import org.apache.spark.storage.RDDInfo -import org.apache.spark.ui.{WebUIPage, UIUtils} +import org.apache.spark.ui.{UIUtils, WebUIPage} import org.apache.spark.util.Utils /** Page showing list of RDD's currently stored in the cluster */ diff --git a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala index 045bd784990d1..0351749700962 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala @@ -35,6 +35,8 @@ private[ui] class StorageTab(parent: SparkUI) extends SparkUITab(parent, "storag /** * :: DeveloperApi :: * A SparkListener that prepares information to be displayed on the BlockManagerUI. + * + * This class is thread-safe (unlike JobProgressListener) */ @DeveloperApi class StorageListener(storageStatusListener: StorageStatusListener) extends SparkListener { @@ -43,7 +45,9 @@ class StorageListener(storageStatusListener: StorageStatusListener) extends Spar def storageStatusList: Seq[StorageStatus] = storageStatusListener.storageStatusList /** Filter RDD info to include only those with cached partitions */ - def rddInfoList: Seq[RDDInfo] = _rddInfoMap.values.filter(_.numCachedPartitions > 0).toSeq + def rddInfoList: Seq[RDDInfo] = synchronized { + _rddInfoMap.values.filter(_.numCachedPartitions > 0).toSeq + } /** Update the storage info of the RDDs whose blocks are among the given updated blocks */ private def updateRDDInfo(updatedBlocks: Seq[(BlockId, BlockStatus)]): Unit = { diff --git a/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json new file mode 100644 index 0000000000000..ce4fe80b66aa5 --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json @@ -0,0 +1,53 @@ +[ { + "id" : "local-1430917381534", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2015-05-06T13:03:00.893GMT", + "endTime" : "2015-05-06T13:03:11.398GMT", + "sparkUser" : "irashid", + "completed" : true + } ] +}, { + "id" : "local-1426533911241", + "name" : "Spark shell", + "attempts" : [ { + "attemptId" : "2", + "startTime" : "2015-03-17T23:11:50.242GMT", + "endTime" : "2015-03-17T23:12:25.177GMT", + "sparkUser" : "irashid", + "completed" : true + }, { + "attemptId" : "1", + "startTime" : "2015-03-16T19:25:10.242GMT", + "endTime" : "2015-03-16T19:25:45.177GMT", + "sparkUser" : "irashid", + "completed" : true + } ] +}, { + "id" : "local-1425081759269", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2015-02-28T00:02:38.277GMT", + "endTime" : "2015-02-28T00:02:46.912GMT", + "sparkUser" : "irashid", + "completed" : true + } ] +}, { + "id" : "local-1422981780767", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2015-02-03T16:42:59.720GMT", + "endTime" : "2015-02-03T16:43:08.731GMT", + "sparkUser" : "irashid", + "completed" : true + } ] +}, { + "id" : "local-1422981759269", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2015-02-03T16:42:38.277GMT", + "endTime" : "2015-02-03T16:42:46.912GMT", + "sparkUser" : "irashid", + "completed" : true + } ] +} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/complete_stage_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/complete_stage_list_json_expectation.json new file mode 100644 index 0000000000000..31ac9beea8788 --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/complete_stage_list_json_expectation.json @@ -0,0 +1,67 @@ +[ { + "status" : "COMPLETE", + "stageId" : 3, + "attemptId" : 0, + "numActiveTasks" : 0, + "numCompleteTasks" : 8, + "numFailedTasks" : 0, + "executorRunTime" : 162, + "inputBytes" : 160, + "inputRecords" : 0, + "outputBytes" : 0, + "outputRecords" : 0, + "shuffleReadBytes" : 0, + "shuffleReadRecords" : 0, + "shuffleWriteBytes" : 0, + "shuffleWriteRecords" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "name" : "count at :17", + "details" : "org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line19.$read$$iwC$$iwC$$iwC$$iwC.(:17)\n$line19.$read$$iwC$$iwC$$iwC.(:22)\n$line19.$read$$iwC$$iwC.(:24)\n$line19.$read$$iwC.(:26)\n$line19.$read.(:28)\n$line19.$read$.(:32)\n$line19.$read$.()\n$line19.$eval$.(:7)\n$line19.$eval$.()\n$line19.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)", + "schedulingPool" : "default", + "accumulatorUpdates" : [ ] +}, { + "status" : "COMPLETE", + "stageId" : 1, + "attemptId" : 0, + "numActiveTasks" : 0, + "numCompleteTasks" : 8, + "numFailedTasks" : 0, + "executorRunTime" : 3476, + "inputBytes" : 28000128, + "inputRecords" : 0, + "outputBytes" : 0, + "outputRecords" : 0, + "shuffleReadBytes" : 0, + "shuffleReadRecords" : 0, + "shuffleWriteBytes" : 13180, + "shuffleWriteRecords" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "name" : "map at :14", + "details" : "org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.(:14)\n$line10.$read$$iwC$$iwC$$iwC.(:19)\n$line10.$read$$iwC$$iwC.(:21)\n$line10.$read$$iwC.(:23)\n$line10.$read.(:25)\n$line10.$read$.(:29)\n$line10.$read$.()\n$line10.$eval$.(:7)\n$line10.$eval$.()\n$line10.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)", + "schedulingPool" : "default", + "accumulatorUpdates" : [ ] +}, { + "status" : "COMPLETE", + "stageId" : 0, + "attemptId" : 0, + "numActiveTasks" : 0, + "numCompleteTasks" : 8, + "numFailedTasks" : 0, + "executorRunTime" : 4338, + "inputBytes" : 0, + "inputRecords" : 0, + "outputBytes" : 0, + "outputRecords" : 0, + "shuffleReadBytes" : 0, + "shuffleReadRecords" : 0, + "shuffleWriteBytes" : 0, + "shuffleWriteRecords" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "name" : "count at :15", + "details" : "org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)", + "schedulingPool" : "default", + "accumulatorUpdates" : [ ] +} ] \ No newline at end of file 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 new file mode 100644 index 0000000000000..ce4fe80b66aa5 --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json @@ -0,0 +1,53 @@ +[ { + "id" : "local-1430917381534", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2015-05-06T13:03:00.893GMT", + "endTime" : "2015-05-06T13:03:11.398GMT", + "sparkUser" : "irashid", + "completed" : true + } ] +}, { + "id" : "local-1426533911241", + "name" : "Spark shell", + "attempts" : [ { + "attemptId" : "2", + "startTime" : "2015-03-17T23:11:50.242GMT", + "endTime" : "2015-03-17T23:12:25.177GMT", + "sparkUser" : "irashid", + "completed" : true + }, { + "attemptId" : "1", + "startTime" : "2015-03-16T19:25:10.242GMT", + "endTime" : "2015-03-16T19:25:45.177GMT", + "sparkUser" : "irashid", + "completed" : true + } ] +}, { + "id" : "local-1425081759269", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2015-02-28T00:02:38.277GMT", + "endTime" : "2015-02-28T00:02:46.912GMT", + "sparkUser" : "irashid", + "completed" : true + } ] +}, { + "id" : "local-1422981780767", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2015-02-03T16:42:59.720GMT", + "endTime" : "2015-02-03T16:43:08.731GMT", + "sparkUser" : "irashid", + "completed" : true + } ] +}, { + "id" : "local-1422981759269", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2015-02-03T16:42:38.277GMT", + "endTime" : "2015-02-03T16:42:46.912GMT", + "sparkUser" : "irashid", + "completed" : true + } ] +} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/executor_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/executor_list_json_expectation.json new file mode 100644 index 0000000000000..cb622e147249e --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/executor_list_json_expectation.json @@ -0,0 +1,17 @@ +[ { + "id" : "", + "hostPort" : "localhost:57971", + "rddBlocks" : 8, + "memoryUsed" : 28000128, + "diskUsed" : 0, + "activeTasks" : 0, + "failedTasks" : 1, + "completedTasks" : 31, + "totalTasks" : 32, + "totalDuration" : 8820, + "totalInputBytes" : 28000288, + "totalShuffleRead" : 0, + "totalShuffleWrite" : 13180, + "maxMemory" : 278302556, + "executorLogs" : { } +} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/failed_stage_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/failed_stage_list_json_expectation.json new file mode 100644 index 0000000000000..bff6a4f69d077 --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/failed_stage_list_json_expectation.json @@ -0,0 +1,23 @@ +[ { + "status" : "FAILED", + "stageId" : 2, + "attemptId" : 0, + "numActiveTasks" : 0, + "numCompleteTasks" : 7, + "numFailedTasks" : 1, + "executorRunTime" : 278, + "inputBytes" : 0, + "inputRecords" : 0, + "outputBytes" : 0, + "outputRecords" : 0, + "shuffleReadBytes" : 0, + "shuffleReadRecords" : 0, + "shuffleWriteBytes" : 0, + "shuffleWriteRecords" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "name" : "count at :20", + "details" : "org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line11.$read$$iwC$$iwC$$iwC$$iwC.(:20)\n$line11.$read$$iwC$$iwC$$iwC.(:25)\n$line11.$read$$iwC$$iwC.(:27)\n$line11.$read$$iwC.(:29)\n$line11.$read.(:31)\n$line11.$read$.(:35)\n$line11.$read$.()\n$line11.$eval$.(:7)\n$line11.$eval$.()\n$line11.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)", + "schedulingPool" : "default", + "accumulatorUpdates" : [ ] +} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/job_list_from_multi_attempt_app_json_1__expectation.json b/core/src/test/resources/HistoryServerExpectations/job_list_from_multi_attempt_app_json_1__expectation.json new file mode 100644 index 0000000000000..2e92e1fa0ec23 --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/job_list_from_multi_attempt_app_json_1__expectation.json @@ -0,0 +1,15 @@ +[ { + "jobId" : 0, + "name" : "foreach at :15", + "stageIds" : [ 0 ], + "status" : "SUCCEEDED", + "numTasks" : 8, + "numActiveTasks" : 0, + "numCompletedTasks" : 8, + "numSkippedTasks" : 8, + "numFailedTasks" : 0, + "numActiveStages" : 0, + "numCompletedStages" : 1, + "numSkippedStages" : 0, + "numFailedStages" : 0 +} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/job_list_from_multi_attempt_app_json_2__expectation.json b/core/src/test/resources/HistoryServerExpectations/job_list_from_multi_attempt_app_json_2__expectation.json new file mode 100644 index 0000000000000..2e92e1fa0ec23 --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/job_list_from_multi_attempt_app_json_2__expectation.json @@ -0,0 +1,15 @@ +[ { + "jobId" : 0, + "name" : "foreach at :15", + "stageIds" : [ 0 ], + "status" : "SUCCEEDED", + "numTasks" : 8, + "numActiveTasks" : 0, + "numCompletedTasks" : 8, + "numSkippedTasks" : 8, + "numFailedTasks" : 0, + "numActiveStages" : 0, + "numCompletedStages" : 1, + "numSkippedStages" : 0, + "numFailedStages" : 0 +} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/job_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/job_list_json_expectation.json new file mode 100644 index 0000000000000..cab4750270dfa --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/job_list_json_expectation.json @@ -0,0 +1,43 @@ +[ { + "jobId" : 2, + "name" : "count at :17", + "stageIds" : [ 3 ], + "status" : "SUCCEEDED", + "numTasks" : 8, + "numActiveTasks" : 0, + "numCompletedTasks" : 8, + "numSkippedTasks" : 8, + "numFailedTasks" : 0, + "numActiveStages" : 0, + "numCompletedStages" : 1, + "numSkippedStages" : 0, + "numFailedStages" : 0 +}, { + "jobId" : 1, + "name" : "count at :20", + "stageIds" : [ 1, 2 ], + "status" : "FAILED", + "numTasks" : 16, + "numActiveTasks" : 0, + "numCompletedTasks" : 15, + "numSkippedTasks" : 15, + "numFailedTasks" : 1, + "numActiveStages" : 0, + "numCompletedStages" : 1, + "numSkippedStages" : 0, + "numFailedStages" : 1 +}, { + "jobId" : 0, + "name" : "count at :15", + "stageIds" : [ 0 ], + "status" : "SUCCEEDED", + "numTasks" : 8, + "numActiveTasks" : 0, + "numCompletedTasks" : 8, + "numSkippedTasks" : 8, + "numFailedTasks" : 0, + "numActiveStages" : 0, + "numCompletedStages" : 1, + "numSkippedStages" : 0, + "numFailedStages" : 0 +} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/maxDate2_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/maxDate2_app_list_json_expectation.json new file mode 100644 index 0000000000000..483632a3956ed --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/maxDate2_app_list_json_expectation.json @@ -0,0 +1,10 @@ +[ { + "id" : "local-1422981759269", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2015-02-03T16:42:38.277GMT", + "endTime" : "2015-02-03T16:42:46.912GMT", + "sparkUser" : "irashid", + "completed" : true + } ] +} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/maxDate_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/maxDate_app_list_json_expectation.json new file mode 100644 index 0000000000000..4b85690fd9199 --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/maxDate_app_list_json_expectation.json @@ -0,0 +1,19 @@ +[ { + "id" : "local-1422981780767", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2015-02-03T16:42:59.720GMT", + "endTime" : "2015-02-03T16:43:08.731GMT", + "sparkUser" : "irashid", + "completed" : true + } ] +}, { + "id" : "local-1422981759269", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2015-02-03T16:42:38.277GMT", + "endTime" : "2015-02-03T16:42:46.912GMT", + "sparkUser" : "irashid", + "completed" : true + } ] +} ] \ No newline at end of file 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 new file mode 100644 index 0000000000000..dca86fe5f7e6a --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/minDate_app_list_json_expectation.json @@ -0,0 +1,35 @@ +[ { + "id" : "local-1430917381534", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2015-05-06T13:03:00.893GMT", + "endTime" : "2015-05-06T13:03:11.398GMT", + "sparkUser" : "irashid", + "completed" : true + } ] +}, { + "id" : "local-1426533911241", + "name" : "Spark shell", + "attempts" : [ { + "attemptId" : "2", + "startTime" : "2015-03-17T23:11:50.242GMT", + "endTime" : "2015-03-17T23:12:25.177GMT", + "sparkUser" : "irashid", + "completed" : true + }, { + "attemptId" : "1", + "startTime" : "2015-03-16T19:25:10.242GMT", + "endTime" : "2015-03-16T19:25:45.177GMT", + "sparkUser" : "irashid", + "completed" : true + } ] +}, { + "id" : "local-1425081759269", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2015-02-28T00:02:38.277GMT", + "endTime" : "2015-02-28T00:02:46.912GMT", + "sparkUser" : "irashid", + "completed" : true + } ] +} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/one_app_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/one_app_json_expectation.json new file mode 100644 index 0000000000000..07489ad96414a --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/one_app_json_expectation.json @@ -0,0 +1,10 @@ +{ + "id" : "local-1422981780767", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2015-02-03T16:42:59.720GMT", + "endTime" : "2015-02-03T16:43:08.731GMT", + "sparkUser" : "irashid", + "completed" : true + } ] +} \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/one_app_multi_attempt_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/one_app_multi_attempt_json_expectation.json new file mode 100644 index 0000000000000..8f3d7160c723f --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/one_app_multi_attempt_json_expectation.json @@ -0,0 +1,17 @@ +{ + "id" : "local-1426533911241", + "name" : "Spark shell", + "attempts" : [ { + "attemptId" : "2", + "startTime" : "2015-03-17T23:11:50.242GMT", + "endTime" : "2015-03-17T23:12:25.177GMT", + "sparkUser" : "irashid", + "completed" : true + }, { + "attemptId" : "1", + "startTime" : "2015-03-16T19:25:10.242GMT", + "endTime" : "2015-03-16T19:25:45.177GMT", + "sparkUser" : "irashid", + "completed" : true + } ] +} \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/one_job_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/one_job_json_expectation.json new file mode 100644 index 0000000000000..4a29072bdb6e4 --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/one_job_json_expectation.json @@ -0,0 +1,15 @@ +{ + "jobId" : 0, + "name" : "count at :15", + "stageIds" : [ 0 ], + "status" : "SUCCEEDED", + "numTasks" : 8, + "numActiveTasks" : 0, + "numCompletedTasks" : 8, + "numSkippedTasks" : 8, + "numFailedTasks" : 0, + "numActiveStages" : 0, + "numCompletedStages" : 1, + "numSkippedStages" : 0, + "numFailedStages" : 0 +} \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/one_rdd_storage_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/one_rdd_storage_json_expectation.json new file mode 100644 index 0000000000000..38b5328ffbb03 --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/one_rdd_storage_json_expectation.json @@ -0,0 +1,64 @@ +{ + "id" : 0, + "name" : "0", + "numPartitions" : 8, + "numCachedPartitions" : 8, + "storageLevel" : "Memory Deserialized 1x Replicated", + "memoryUsed" : 28000128, + "diskUsed" : 0, + "dataDistribution" : [ { + "address" : "localhost:57971", + "memoryUsed" : 28000128, + "memoryRemaining" : 250302428, + "diskUsed" : 0 + } ], + "partitions" : [ { + "blockName" : "rdd_0_0", + "storageLevel" : "Memory Deserialized 1x Replicated", + "memoryUsed" : 3500016, + "diskUsed" : 0, + "executors" : [ "localhost:57971" ] + }, { + "blockName" : "rdd_0_1", + "storageLevel" : "Memory Deserialized 1x Replicated", + "memoryUsed" : 3500016, + "diskUsed" : 0, + "executors" : [ "localhost:57971" ] + }, { + "blockName" : "rdd_0_2", + "storageLevel" : "Memory Deserialized 1x Replicated", + "memoryUsed" : 3500016, + "diskUsed" : 0, + "executors" : [ "localhost:57971" ] + }, { + "blockName" : "rdd_0_3", + "storageLevel" : "Memory Deserialized 1x Replicated", + "memoryUsed" : 3500016, + "diskUsed" : 0, + "executors" : [ "localhost:57971" ] + }, { + "blockName" : "rdd_0_4", + "storageLevel" : "Memory Deserialized 1x Replicated", + "memoryUsed" : 3500016, + "diskUsed" : 0, + "executors" : [ "localhost:57971" ] + }, { + "blockName" : "rdd_0_5", + "storageLevel" : "Memory Deserialized 1x Replicated", + "memoryUsed" : 3500016, + "diskUsed" : 0, + "executors" : [ "localhost:57971" ] + }, { + "blockName" : "rdd_0_6", + "storageLevel" : "Memory Deserialized 1x Replicated", + "memoryUsed" : 3500016, + "diskUsed" : 0, + "executors" : [ "localhost:57971" ] + }, { + "blockName" : "rdd_0_7", + "storageLevel" : "Memory Deserialized 1x Replicated", + "memoryUsed" : 3500016, + "diskUsed" : 0, + "executors" : [ "localhost:57971" ] + } ] +} \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/one_stage_attempt_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/one_stage_attempt_json_expectation.json new file mode 100644 index 0000000000000..111cb8163eb3d --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/one_stage_attempt_json_expectation.json @@ -0,0 +1,270 @@ +{ + "status" : "COMPLETE", + "stageId" : 1, + "attemptId" : 0, + "numActiveTasks" : 0, + "numCompleteTasks" : 8, + "numFailedTasks" : 0, + "executorRunTime" : 3476, + "inputBytes" : 28000128, + "inputRecords" : 0, + "outputBytes" : 0, + "outputRecords" : 0, + "shuffleReadBytes" : 0, + "shuffleReadRecords" : 0, + "shuffleWriteBytes" : 13180, + "shuffleWriteRecords" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "name" : "map at :14", + "details" : "org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.(:14)\n$line10.$read$$iwC$$iwC$$iwC.(:19)\n$line10.$read$$iwC$$iwC.(:21)\n$line10.$read$$iwC.(:23)\n$line10.$read.(:25)\n$line10.$read$.(:29)\n$line10.$read$.()\n$line10.$eval$.(:7)\n$line10.$eval$.()\n$line10.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)", + "schedulingPool" : "default", + "accumulatorUpdates" : [ ], + "tasks" : { + "8" : { + "taskId" : 8, + "index" : 0, + "attempt" : 0, + "launchTime" : "2015-02-03T16:43:05.829GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 1, + "executorRunTime" : 435, + "resultSize" : 1902, + "jvmGcTime" : 19, + "resultSerializationTime" : 2, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 3500016, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1648, + "writeTime" : 94000, + "recordsWritten" : 0 + } + } + }, + "11" : { + "taskId" : 11, + "index" : 3, + "attempt" : 0, + "launchTime" : "2015-02-03T16:43:05.830GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 2, + "executorRunTime" : 434, + "resultSize" : 1902, + "jvmGcTime" : 19, + "resultSerializationTime" : 1, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 3500016, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1647, + "writeTime" : 83000, + "recordsWritten" : 0 + } + } + }, + "14" : { + "taskId" : 14, + "index" : 6, + "attempt" : 0, + "launchTime" : "2015-02-03T16:43:05.832GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 2, + "executorRunTime" : 434, + "resultSize" : 1902, + "jvmGcTime" : 19, + "resultSerializationTime" : 1, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 3500016, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1648, + "writeTime" : 88000, + "recordsWritten" : 0 + } + } + }, + "13" : { + "taskId" : 13, + "index" : 5, + "attempt" : 0, + "launchTime" : "2015-02-03T16:43:05.831GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 2, + "executorRunTime" : 434, + "resultSize" : 1902, + "jvmGcTime" : 19, + "resultSerializationTime" : 2, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 3500016, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1648, + "writeTime" : 73000, + "recordsWritten" : 0 + } + } + }, + "10" : { + "taskId" : 10, + "index" : 2, + "attempt" : 0, + "launchTime" : "2015-02-03T16:43:05.830GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 2, + "executorRunTime" : 434, + "resultSize" : 1902, + "jvmGcTime" : 19, + "resultSerializationTime" : 1, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 3500016, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1648, + "writeTime" : 76000, + "recordsWritten" : 0 + } + } + }, + "9" : { + "taskId" : 9, + "index" : 1, + "attempt" : 0, + "launchTime" : "2015-02-03T16:43:05.830GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 1, + "executorRunTime" : 436, + "resultSize" : 1902, + "jvmGcTime" : 19, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 3500016, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1648, + "writeTime" : 98000, + "recordsWritten" : 0 + } + } + }, + "12" : { + "taskId" : 12, + "index" : 4, + "attempt" : 0, + "launchTime" : "2015-02-03T16:43:05.831GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 2, + "executorRunTime" : 434, + "resultSize" : 1902, + "jvmGcTime" : 19, + "resultSerializationTime" : 1, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 3500016, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1645, + "writeTime" : 101000, + "recordsWritten" : 0 + } + } + }, + "15" : { + "taskId" : 15, + "index" : 7, + "attempt" : 0, + "launchTime" : "2015-02-03T16:43:05.833GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 1, + "executorRunTime" : 435, + "resultSize" : 1902, + "jvmGcTime" : 19, + "resultSerializationTime" : 1, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 3500016, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1648, + "writeTime" : 79000, + "recordsWritten" : 0 + } + } + } + }, + "executorSummary" : { + "" : { + "taskTime" : 3624, + "failedTasks" : 0, + "succeededTasks" : 8, + "inputBytes" : 28000128, + "outputBytes" : 0, + "shuffleRead" : 0, + "shuffleWrite" : 13180, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0 + } + } +} \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/one_stage_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/one_stage_json_expectation.json new file mode 100644 index 0000000000000..ef339f89afa45 --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/one_stage_json_expectation.json @@ -0,0 +1,270 @@ +[ { + "status" : "COMPLETE", + "stageId" : 1, + "attemptId" : 0, + "numActiveTasks" : 0, + "numCompleteTasks" : 8, + "numFailedTasks" : 0, + "executorRunTime" : 3476, + "inputBytes" : 28000128, + "inputRecords" : 0, + "outputBytes" : 0, + "outputRecords" : 0, + "shuffleReadBytes" : 0, + "shuffleReadRecords" : 0, + "shuffleWriteBytes" : 13180, + "shuffleWriteRecords" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "name" : "map at :14", + "details" : "org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.(:14)\n$line10.$read$$iwC$$iwC$$iwC.(:19)\n$line10.$read$$iwC$$iwC.(:21)\n$line10.$read$$iwC.(:23)\n$line10.$read.(:25)\n$line10.$read$.(:29)\n$line10.$read$.()\n$line10.$eval$.(:7)\n$line10.$eval$.()\n$line10.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)", + "schedulingPool" : "default", + "accumulatorUpdates" : [ ], + "tasks" : { + "8" : { + "taskId" : 8, + "index" : 0, + "attempt" : 0, + "launchTime" : "2015-02-03T16:43:05.829GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 1, + "executorRunTime" : 435, + "resultSize" : 1902, + "jvmGcTime" : 19, + "resultSerializationTime" : 2, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 3500016, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1648, + "writeTime" : 94000, + "recordsWritten" : 0 + } + } + }, + "11" : { + "taskId" : 11, + "index" : 3, + "attempt" : 0, + "launchTime" : "2015-02-03T16:43:05.830GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 2, + "executorRunTime" : 434, + "resultSize" : 1902, + "jvmGcTime" : 19, + "resultSerializationTime" : 1, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 3500016, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1647, + "writeTime" : 83000, + "recordsWritten" : 0 + } + } + }, + "14" : { + "taskId" : 14, + "index" : 6, + "attempt" : 0, + "launchTime" : "2015-02-03T16:43:05.832GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 2, + "executorRunTime" : 434, + "resultSize" : 1902, + "jvmGcTime" : 19, + "resultSerializationTime" : 1, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 3500016, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1648, + "writeTime" : 88000, + "recordsWritten" : 0 + } + } + }, + "13" : { + "taskId" : 13, + "index" : 5, + "attempt" : 0, + "launchTime" : "2015-02-03T16:43:05.831GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 2, + "executorRunTime" : 434, + "resultSize" : 1902, + "jvmGcTime" : 19, + "resultSerializationTime" : 2, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 3500016, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1648, + "writeTime" : 73000, + "recordsWritten" : 0 + } + } + }, + "10" : { + "taskId" : 10, + "index" : 2, + "attempt" : 0, + "launchTime" : "2015-02-03T16:43:05.830GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 2, + "executorRunTime" : 434, + "resultSize" : 1902, + "jvmGcTime" : 19, + "resultSerializationTime" : 1, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 3500016, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1648, + "writeTime" : 76000, + "recordsWritten" : 0 + } + } + }, + "9" : { + "taskId" : 9, + "index" : 1, + "attempt" : 0, + "launchTime" : "2015-02-03T16:43:05.830GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 1, + "executorRunTime" : 436, + "resultSize" : 1902, + "jvmGcTime" : 19, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 3500016, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1648, + "writeTime" : 98000, + "recordsWritten" : 0 + } + } + }, + "12" : { + "taskId" : 12, + "index" : 4, + "attempt" : 0, + "launchTime" : "2015-02-03T16:43:05.831GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 2, + "executorRunTime" : 434, + "resultSize" : 1902, + "jvmGcTime" : 19, + "resultSerializationTime" : 1, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 3500016, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1645, + "writeTime" : 101000, + "recordsWritten" : 0 + } + } + }, + "15" : { + "taskId" : 15, + "index" : 7, + "attempt" : 0, + "launchTime" : "2015-02-03T16:43:05.833GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 1, + "executorRunTime" : 435, + "resultSize" : 1902, + "jvmGcTime" : 19, + "resultSerializationTime" : 1, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 3500016, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1648, + "writeTime" : 79000, + "recordsWritten" : 0 + } + } + } + }, + "executorSummary" : { + "" : { + "taskTime" : 3624, + "failedTasks" : 0, + "succeededTasks" : 8, + "inputBytes" : 28000128, + "outputBytes" : 0, + "shuffleRead" : 0, + "shuffleWrite" : 13180, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0 + } + } +} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/rdd_list_storage_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/rdd_list_storage_json_expectation.json new file mode 100644 index 0000000000000..f79a31022d214 --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/rdd_list_storage_json_expectation.json @@ -0,0 +1,9 @@ +[ { + "id" : 0, + "name" : "0", + "numPartitions" : 8, + "numCachedPartitions" : 8, + "storageLevel" : "Memory Deserialized 1x Replicated", + "memoryUsed" : 28000128, + "diskUsed" : 0 +} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/running_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/running_app_list_json_expectation.json new file mode 100644 index 0000000000000..8878e547a7984 --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/running_app_list_json_expectation.json @@ -0,0 +1 @@ +[ ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/stage_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_list_json_expectation.json new file mode 100644 index 0000000000000..056fac7088594 --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/stage_list_json_expectation.json @@ -0,0 +1,89 @@ +[ { + "status" : "COMPLETE", + "stageId" : 3, + "attemptId" : 0, + "numActiveTasks" : 0, + "numCompleteTasks" : 8, + "numFailedTasks" : 0, + "executorRunTime" : 162, + "inputBytes" : 160, + "inputRecords" : 0, + "outputBytes" : 0, + "outputRecords" : 0, + "shuffleReadBytes" : 0, + "shuffleReadRecords" : 0, + "shuffleWriteBytes" : 0, + "shuffleWriteRecords" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "name" : "count at :17", + "details" : "org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line19.$read$$iwC$$iwC$$iwC$$iwC.(:17)\n$line19.$read$$iwC$$iwC$$iwC.(:22)\n$line19.$read$$iwC$$iwC.(:24)\n$line19.$read$$iwC.(:26)\n$line19.$read.(:28)\n$line19.$read$.(:32)\n$line19.$read$.()\n$line19.$eval$.(:7)\n$line19.$eval$.()\n$line19.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)", + "schedulingPool" : "default", + "accumulatorUpdates" : [ ] +}, { + "status" : "COMPLETE", + "stageId" : 1, + "attemptId" : 0, + "numActiveTasks" : 0, + "numCompleteTasks" : 8, + "numFailedTasks" : 0, + "executorRunTime" : 3476, + "inputBytes" : 28000128, + "inputRecords" : 0, + "outputBytes" : 0, + "outputRecords" : 0, + "shuffleReadBytes" : 0, + "shuffleReadRecords" : 0, + "shuffleWriteBytes" : 13180, + "shuffleWriteRecords" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "name" : "map at :14", + "details" : "org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.(:14)\n$line10.$read$$iwC$$iwC$$iwC.(:19)\n$line10.$read$$iwC$$iwC.(:21)\n$line10.$read$$iwC.(:23)\n$line10.$read.(:25)\n$line10.$read$.(:29)\n$line10.$read$.()\n$line10.$eval$.(:7)\n$line10.$eval$.()\n$line10.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)", + "schedulingPool" : "default", + "accumulatorUpdates" : [ ] +}, { + "status" : "COMPLETE", + "stageId" : 0, + "attemptId" : 0, + "numActiveTasks" : 0, + "numCompleteTasks" : 8, + "numFailedTasks" : 0, + "executorRunTime" : 4338, + "inputBytes" : 0, + "inputRecords" : 0, + "outputBytes" : 0, + "outputRecords" : 0, + "shuffleReadBytes" : 0, + "shuffleReadRecords" : 0, + "shuffleWriteBytes" : 0, + "shuffleWriteRecords" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "name" : "count at :15", + "details" : "org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)", + "schedulingPool" : "default", + "accumulatorUpdates" : [ ] +}, { + "status" : "FAILED", + "stageId" : 2, + "attemptId" : 0, + "numActiveTasks" : 0, + "numCompleteTasks" : 7, + "numFailedTasks" : 1, + "executorRunTime" : 278, + "inputBytes" : 0, + "inputRecords" : 0, + "outputBytes" : 0, + "outputRecords" : 0, + "shuffleReadBytes" : 0, + "shuffleReadRecords" : 0, + "shuffleWriteBytes" : 0, + "shuffleWriteRecords" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "name" : "count at :20", + "details" : "org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line11.$read$$iwC$$iwC$$iwC$$iwC.(:20)\n$line11.$read$$iwC$$iwC$$iwC.(:25)\n$line11.$read$$iwC$$iwC.(:27)\n$line11.$read$$iwC.(:29)\n$line11.$read.(:31)\n$line11.$read$.(:35)\n$line11.$read$.()\n$line11.$eval$.(:7)\n$line11.$eval$.()\n$line11.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)", + "schedulingPool" : "default", + "accumulatorUpdates" : [ ] +} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/stage_list_with_accumulable_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_list_with_accumulable_json_expectation.json new file mode 100644 index 0000000000000..79ccacd309693 --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/stage_list_with_accumulable_json_expectation.json @@ -0,0 +1,27 @@ +[ { + "status" : "COMPLETE", + "stageId" : 0, + "attemptId" : 0, + "numActiveTasks" : 0, + "numCompleteTasks" : 8, + "numFailedTasks" : 0, + "executorRunTime" : 120, + "inputBytes" : 0, + "inputRecords" : 0, + "outputBytes" : 0, + "outputRecords" : 0, + "shuffleReadBytes" : 0, + "shuffleReadRecords" : 0, + "shuffleWriteBytes" : 0, + "shuffleWriteRecords" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "name" : "foreach at :15", + "details" : "org.apache.spark.rdd.RDD.foreach(RDD.scala:765)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)", + "schedulingPool" : "default", + "accumulatorUpdates" : [ { + "id" : 1, + "name" : "my counter", + "value" : "5050" + } ] +} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_list_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_list_expectation.json new file mode 100644 index 0000000000000..f2cb29b31c85f --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/stage_task_list_expectation.json @@ -0,0 +1,561 @@ +[ { + "taskId" : 0, + "index" : 0, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:06.494GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 32, + "executorRunTime" : 349, + "resultSize" : 2010, + "jvmGcTime" : 7, + "resultSerializationTime" : 1, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 49294, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 3842811, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 1, + "index" : 1, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:06.502GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 31, + "executorRunTime" : 350, + "resultSize" : 2010, + "jvmGcTime" : 7, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 60488, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 3934399, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 2, + "index" : 2, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:06.503GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 32, + "executorRunTime" : 348, + "resultSize" : 2010, + "jvmGcTime" : 7, + "resultSerializationTime" : 2, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 60488, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 89885, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 3, + "index" : 3, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:06.504GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 31, + "executorRunTime" : 349, + "resultSize" : 2010, + "jvmGcTime" : 7, + "resultSerializationTime" : 2, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 60488, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 1311694, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 4, + "index" : 4, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:06.504GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 31, + "executorRunTime" : 349, + "resultSize" : 2010, + "jvmGcTime" : 7, + "resultSerializationTime" : 1, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 60488, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 83022, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 5, + "index" : 5, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:06.505GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 30, + "executorRunTime" : 350, + "resultSize" : 2010, + "jvmGcTime" : 7, + "resultSerializationTime" : 1, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 60488, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 3675510, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 6, + "index" : 6, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:06.505GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 29, + "executorRunTime" : 351, + "resultSize" : 2010, + "jvmGcTime" : 7, + "resultSerializationTime" : 1, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 60488, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 4016617, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 7, + "index" : 7, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:06.506GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 31, + "executorRunTime" : 349, + "resultSize" : 2010, + "jvmGcTime" : 7, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 60488, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 2579051, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 8, + "index" : 8, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:06.914GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 3, + "executorRunTime" : 80, + "resultSize" : 2010, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 60488, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 121551, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 9, + "index" : 9, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:06.915GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 9, + "executorRunTime" : 84, + "resultSize" : 2010, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 60489, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 101664, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 10, + "index" : 10, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:06.916GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 8, + "executorRunTime" : 73, + "resultSize" : 2010, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 94709, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 11, + "index" : 11, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:06.918GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 6, + "executorRunTime" : 75, + "resultSize" : 2010, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 94507, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 12, + "index" : 12, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:06.923GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 9, + "executorRunTime" : 77, + "resultSize" : 2010, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 102476, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 13, + "index" : 13, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:06.924GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 9, + "executorRunTime" : 76, + "resultSize" : 2010, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 95004, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 14, + "index" : 14, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:06.925GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 6, + "executorRunTime" : 83, + "resultSize" : 2010, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 95646, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 15, + "index" : 15, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:06.928GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 3, + "executorRunTime" : 76, + "resultSize" : 2010, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 602780, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 16, + "index" : 16, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.001GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 10, + "executorRunTime" : 84, + "resultSize" : 2010, + "jvmGcTime" : 5, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 108320, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 17, + "index" : 17, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.005GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 11, + "executorRunTime" : 91, + "resultSize" : 2065, + "jvmGcTime" : 5, + "resultSerializationTime" : 1, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 99944, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 18, + "index" : 18, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.010GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 10, + "executorRunTime" : 92, + "resultSize" : 2065, + "jvmGcTime" : 5, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 100836, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 19, + "index" : 19, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.012GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 5, + "executorRunTime" : 84, + "resultSize" : 2010, + "jvmGcTime" : 5, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 95788, + "recordsWritten" : 10 + } + } +} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_list_from_multi_attempt_app_json_1__expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_list_from_multi_attempt_app_json_1__expectation.json new file mode 100644 index 0000000000000..c3febc5fc9447 --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/stage_task_list_from_multi_attempt_app_json_1__expectation.json @@ -0,0 +1,193 @@ +[ { + "taskId" : 0, + "index" : 0, + "attempt" : 0, + "launchTime" : "2015-03-16T19:25:36.515GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ { + "id" : 1, + "name" : "my counter", + "update" : "78", + "value" : "5050" + } ], + "taskMetrics" : { + "executorDeserializeTime" : 14, + "executorRunTime" : 15, + "resultSize" : 697, + "jvmGcTime" : 0, + "resultSerializationTime" : 2, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0 + } +}, { + "taskId" : 1, + "index" : 1, + "attempt" : 0, + "launchTime" : "2015-03-16T19:25:36.521GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ { + "id" : 1, + "name" : "my counter", + "update" : "247", + "value" : "2175" + } ], + "taskMetrics" : { + "executorDeserializeTime" : 14, + "executorRunTime" : 15, + "resultSize" : 697, + "jvmGcTime" : 0, + "resultSerializationTime" : 2, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0 + } +}, { + "taskId" : 2, + "index" : 2, + "attempt" : 0, + "launchTime" : "2015-03-16T19:25:36.522GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ { + "id" : 1, + "name" : "my counter", + "update" : "378", + "value" : "378" + } ], + "taskMetrics" : { + "executorDeserializeTime" : 13, + "executorRunTime" : 15, + "resultSize" : 697, + "jvmGcTime" : 0, + "resultSerializationTime" : 2, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0 + } +}, { + "taskId" : 3, + "index" : 3, + "attempt" : 0, + "launchTime" : "2015-03-16T19:25:36.522GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ { + "id" : 1, + "name" : "my counter", + "update" : "572", + "value" : "950" + } ], + "taskMetrics" : { + "executorDeserializeTime" : 13, + "executorRunTime" : 15, + "resultSize" : 697, + "jvmGcTime" : 0, + "resultSerializationTime" : 2, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0 + } +}, { + "taskId" : 4, + "index" : 4, + "attempt" : 0, + "launchTime" : "2015-03-16T19:25:36.522GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ { + "id" : 1, + "name" : "my counter", + "update" : "678", + "value" : "2853" + } ], + "taskMetrics" : { + "executorDeserializeTime" : 12, + "executorRunTime" : 15, + "resultSize" : 697, + "jvmGcTime" : 0, + "resultSerializationTime" : 1, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0 + } +}, { + "taskId" : 5, + "index" : 5, + "attempt" : 0, + "launchTime" : "2015-03-16T19:25:36.523GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ { + "id" : 1, + "name" : "my counter", + "update" : "897", + "value" : "3750" + } ], + "taskMetrics" : { + "executorDeserializeTime" : 12, + "executorRunTime" : 15, + "resultSize" : 697, + "jvmGcTime" : 0, + "resultSerializationTime" : 2, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0 + } +}, { + "taskId" : 6, + "index" : 6, + "attempt" : 0, + "launchTime" : "2015-03-16T19:25:36.523GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ { + "id" : 1, + "name" : "my counter", + "update" : "978", + "value" : "1928" + } ], + "taskMetrics" : { + "executorDeserializeTime" : 12, + "executorRunTime" : 15, + "resultSize" : 697, + "jvmGcTime" : 0, + "resultSerializationTime" : 2, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0 + } +}, { + "taskId" : 7, + "index" : 7, + "attempt" : 0, + "launchTime" : "2015-03-16T19:25:36.524GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ { + "id" : 1, + "name" : "my counter", + "update" : "1222", + "value" : "4972" + } ], + "taskMetrics" : { + "executorDeserializeTime" : 12, + "executorRunTime" : 15, + "resultSize" : 697, + "jvmGcTime" : 0, + "resultSerializationTime" : 2, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0 + } +} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_list_from_multi_attempt_app_json_2__expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_list_from_multi_attempt_app_json_2__expectation.json new file mode 100644 index 0000000000000..56d667d88917c --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/stage_task_list_from_multi_attempt_app_json_2__expectation.json @@ -0,0 +1,193 @@ +[ { + "taskId" : 0, + "index" : 0, + "attempt" : 0, + "launchTime" : "2015-03-17T23:12:16.515GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ { + "id" : 1, + "name" : "my counter", + "update" : "78", + "value" : "5050" + } ], + "taskMetrics" : { + "executorDeserializeTime" : 14, + "executorRunTime" : 15, + "resultSize" : 697, + "jvmGcTime" : 0, + "resultSerializationTime" : 2, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0 + } +}, { + "taskId" : 1, + "index" : 1, + "attempt" : 0, + "launchTime" : "2015-03-17T23:12:16.521GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ { + "id" : 1, + "name" : "my counter", + "update" : "247", + "value" : "2175" + } ], + "taskMetrics" : { + "executorDeserializeTime" : 14, + "executorRunTime" : 15, + "resultSize" : 697, + "jvmGcTime" : 0, + "resultSerializationTime" : 2, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0 + } +}, { + "taskId" : 2, + "index" : 2, + "attempt" : 0, + "launchTime" : "2015-03-17T23:12:16.522GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ { + "id" : 1, + "name" : "my counter", + "update" : "378", + "value" : "378" + } ], + "taskMetrics" : { + "executorDeserializeTime" : 13, + "executorRunTime" : 15, + "resultSize" : 697, + "jvmGcTime" : 0, + "resultSerializationTime" : 2, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0 + } +}, { + "taskId" : 3, + "index" : 3, + "attempt" : 0, + "launchTime" : "2015-03-17T23:12:16.522GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ { + "id" : 1, + "name" : "my counter", + "update" : "572", + "value" : "950" + } ], + "taskMetrics" : { + "executorDeserializeTime" : 13, + "executorRunTime" : 15, + "resultSize" : 697, + "jvmGcTime" : 0, + "resultSerializationTime" : 2, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0 + } +}, { + "taskId" : 4, + "index" : 4, + "attempt" : 0, + "launchTime" : "2015-03-17T23:12:16.522GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ { + "id" : 1, + "name" : "my counter", + "update" : "678", + "value" : "2853" + } ], + "taskMetrics" : { + "executorDeserializeTime" : 12, + "executorRunTime" : 15, + "resultSize" : 697, + "jvmGcTime" : 0, + "resultSerializationTime" : 1, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0 + } +}, { + "taskId" : 5, + "index" : 5, + "attempt" : 0, + "launchTime" : "2015-03-17T23:12:16.523GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ { + "id" : 1, + "name" : "my counter", + "update" : "897", + "value" : "3750" + } ], + "taskMetrics" : { + "executorDeserializeTime" : 12, + "executorRunTime" : 15, + "resultSize" : 697, + "jvmGcTime" : 0, + "resultSerializationTime" : 2, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0 + } +}, { + "taskId" : 6, + "index" : 6, + "attempt" : 0, + "launchTime" : "2015-03-17T23:12:16.523GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ { + "id" : 1, + "name" : "my counter", + "update" : "978", + "value" : "1928" + } ], + "taskMetrics" : { + "executorDeserializeTime" : 12, + "executorRunTime" : 15, + "resultSize" : 697, + "jvmGcTime" : 0, + "resultSerializationTime" : 2, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0 + } +}, { + "taskId" : 7, + "index" : 7, + "attempt" : 0, + "launchTime" : "2015-03-17T23:12:16.524GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ { + "id" : 1, + "name" : "my counter", + "update" : "1222", + "value" : "4972" + } ], + "taskMetrics" : { + "executorDeserializeTime" : 12, + "executorRunTime" : 15, + "resultSize" : 697, + "jvmGcTime" : 0, + "resultSerializationTime" : 2, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0 + } +} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__offset___length_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__offset___length_expectation.json new file mode 100644 index 0000000000000..e5ec3bc4c7126 --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__offset___length_expectation.json @@ -0,0 +1,1401 @@ +[ { + "taskId" : 10, + "index" : 10, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:06.916GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 8, + "executorRunTime" : 73, + "resultSize" : 2010, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 94709, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 11, + "index" : 11, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:06.918GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 6, + "executorRunTime" : 75, + "resultSize" : 2010, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 94507, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 12, + "index" : 12, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:06.923GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 9, + "executorRunTime" : 77, + "resultSize" : 2010, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 102476, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 13, + "index" : 13, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:06.924GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 9, + "executorRunTime" : 76, + "resultSize" : 2010, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 95004, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 14, + "index" : 14, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:06.925GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 6, + "executorRunTime" : 83, + "resultSize" : 2010, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 95646, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 15, + "index" : 15, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:06.928GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 3, + "executorRunTime" : 76, + "resultSize" : 2010, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 602780, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 16, + "index" : 16, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.001GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 10, + "executorRunTime" : 84, + "resultSize" : 2010, + "jvmGcTime" : 5, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 108320, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 17, + "index" : 17, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.005GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 11, + "executorRunTime" : 91, + "resultSize" : 2065, + "jvmGcTime" : 5, + "resultSerializationTime" : 1, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 99944, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 18, + "index" : 18, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.010GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 10, + "executorRunTime" : 92, + "resultSize" : 2065, + "jvmGcTime" : 5, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 100836, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 19, + "index" : 19, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.012GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 5, + "executorRunTime" : 84, + "resultSize" : 2010, + "jvmGcTime" : 5, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 95788, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 20, + "index" : 20, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.014GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 3, + "executorRunTime" : 83, + "resultSize" : 2010, + "jvmGcTime" : 5, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 97716, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 21, + "index" : 21, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.015GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 5, + "executorRunTime" : 88, + "resultSize" : 2010, + "jvmGcTime" : 5, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 100270, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 22, + "index" : 22, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.018GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 5, + "executorRunTime" : 93, + "resultSize" : 2065, + "jvmGcTime" : 5, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 143427, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 23, + "index" : 23, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.031GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 3, + "executorRunTime" : 65, + "resultSize" : 2010, + "jvmGcTime" : 5, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 91844, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 24, + "index" : 24, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.098GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 3, + "executorRunTime" : 43, + "resultSize" : 2065, + "jvmGcTime" : 0, + "resultSerializationTime" : 1, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 157194, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 25, + "index" : 25, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.103GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 3, + "executorRunTime" : 49, + "resultSize" : 2065, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 94134, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 26, + "index" : 26, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.105GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 6, + "executorRunTime" : 38, + "resultSize" : 2065, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 108213, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 27, + "index" : 27, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.110GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 3, + "executorRunTime" : 32, + "resultSize" : 2065, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 102019, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 28, + "index" : 28, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.113GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 3, + "executorRunTime" : 29, + "resultSize" : 2065, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 104299, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 29, + "index" : 29, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.114GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 2, + "executorRunTime" : 39, + "resultSize" : 2065, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 114938, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 30, + "index" : 30, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.118GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 4, + "executorRunTime" : 34, + "resultSize" : 2065, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 119770, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 31, + "index" : 31, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.127GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 36, + "executorRunTime" : 24, + "resultSize" : 2065, + "jvmGcTime" : 5, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 92619, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 32, + "index" : 32, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.148GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 3, + "executorRunTime" : 17, + "resultSize" : 2065, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 89603, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 33, + "index" : 33, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.149GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 5, + "executorRunTime" : 43, + "resultSize" : 2065, + "jvmGcTime" : 5, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 118329, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 34, + "index" : 34, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.156GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 6, + "executorRunTime" : 27, + "resultSize" : 2065, + "jvmGcTime" : 5, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 127746, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 35, + "index" : 35, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.161GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 5, + "executorRunTime" : 35, + "resultSize" : 2065, + "jvmGcTime" : 5, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 160963, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 36, + "index" : 36, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.164GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 5, + "executorRunTime" : 29, + "resultSize" : 2065, + "jvmGcTime" : 5, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 123855, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 37, + "index" : 37, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.165GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 4, + "executorRunTime" : 32, + "resultSize" : 2065, + "jvmGcTime" : 5, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 111869, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 38, + "index" : 38, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.166GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 3, + "executorRunTime" : 31, + "resultSize" : 2065, + "jvmGcTime" : 5, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 131158, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 39, + "index" : 39, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.180GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 2, + "executorRunTime" : 17, + "resultSize" : 2065, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 98748, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 40, + "index" : 40, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.197GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 4, + "executorRunTime" : 14, + "resultSize" : 2065, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 94792, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 41, + "index" : 41, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.200GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 2, + "executorRunTime" : 16, + "resultSize" : 2065, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 90765, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 42, + "index" : 42, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.203GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 10, + "executorRunTime" : 17, + "resultSize" : 2065, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 103713, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 43, + "index" : 43, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.204GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 2, + "executorRunTime" : 16, + "resultSize" : 2065, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 171516, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 44, + "index" : 44, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.205GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 3, + "executorRunTime" : 18, + "resultSize" : 2065, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 98293, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 45, + "index" : 45, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.206GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 2, + "executorRunTime" : 19, + "resultSize" : 2065, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 92985, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 46, + "index" : 46, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.210GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 1, + "executorRunTime" : 31, + "resultSize" : 2065, + "jvmGcTime" : 6, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 113322, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 47, + "index" : 47, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.212GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 2, + "executorRunTime" : 18, + "resultSize" : 2065, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 103015, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 48, + "index" : 48, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.220GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 3, + "executorRunTime" : 24, + "resultSize" : 2065, + "jvmGcTime" : 6, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 139844, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 49, + "index" : 49, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.223GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 7, + "executorRunTime" : 23, + "resultSize" : 2065, + "jvmGcTime" : 6, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 94984, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 50, + "index" : 50, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.240GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 4, + "executorRunTime" : 18, + "resultSize" : 2065, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 90836, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 51, + "index" : 51, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.242GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 2, + "executorRunTime" : 17, + "resultSize" : 2065, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 96013, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 52, + "index" : 52, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.243GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 5, + "executorRunTime" : 18, + "resultSize" : 2065, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 89664, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 53, + "index" : 53, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.244GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 6, + "executorRunTime" : 18, + "resultSize" : 2065, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 92835, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 54, + "index" : 54, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.244GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 3, + "executorRunTime" : 18, + "resultSize" : 2065, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 90506, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 55, + "index" : 55, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.246GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 4, + "executorRunTime" : 21, + "resultSize" : 2065, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 108309, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 56, + "index" : 56, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.249GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 5, + "executorRunTime" : 20, + "resultSize" : 2065, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 90329, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 57, + "index" : 57, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.257GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 3, + "executorRunTime" : 16, + "resultSize" : 2065, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 96849, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 58, + "index" : 58, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.263GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 3, + "executorRunTime" : 16, + "resultSize" : 2065, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 97521, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 59, + "index" : 59, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.265GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 3, + "executorRunTime" : 17, + "resultSize" : 2065, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 100753, + "recordsWritten" : 10 + } + } +} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_expectation.json new file mode 100644 index 0000000000000..5657123a2db15 --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_expectation.json @@ -0,0 +1,561 @@ +[ { + "taskId" : 6, + "index" : 6, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:06.505GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 29, + "executorRunTime" : 351, + "resultSize" : 2010, + "jvmGcTime" : 7, + "resultSerializationTime" : 1, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 60488, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 4016617, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 5, + "index" : 5, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:06.505GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 30, + "executorRunTime" : 350, + "resultSize" : 2010, + "jvmGcTime" : 7, + "resultSerializationTime" : 1, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 60488, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 3675510, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 1, + "index" : 1, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:06.502GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 31, + "executorRunTime" : 350, + "resultSize" : 2010, + "jvmGcTime" : 7, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 60488, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 3934399, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 4, + "index" : 4, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:06.504GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 31, + "executorRunTime" : 349, + "resultSize" : 2010, + "jvmGcTime" : 7, + "resultSerializationTime" : 1, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 60488, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 83022, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 7, + "index" : 7, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:06.506GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 31, + "executorRunTime" : 349, + "resultSize" : 2010, + "jvmGcTime" : 7, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 60488, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 2579051, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 3, + "index" : 3, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:06.504GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 31, + "executorRunTime" : 349, + "resultSize" : 2010, + "jvmGcTime" : 7, + "resultSerializationTime" : 2, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 60488, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 1311694, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 0, + "index" : 0, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:06.494GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 32, + "executorRunTime" : 349, + "resultSize" : 2010, + "jvmGcTime" : 7, + "resultSerializationTime" : 1, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 49294, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 3842811, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 2, + "index" : 2, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:06.503GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 32, + "executorRunTime" : 348, + "resultSize" : 2010, + "jvmGcTime" : 7, + "resultSerializationTime" : 2, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 60488, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 89885, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 22, + "index" : 22, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.018GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 5, + "executorRunTime" : 93, + "resultSize" : 2065, + "jvmGcTime" : 5, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 143427, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 18, + "index" : 18, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.010GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 10, + "executorRunTime" : 92, + "resultSize" : 2065, + "jvmGcTime" : 5, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 100836, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 17, + "index" : 17, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.005GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 11, + "executorRunTime" : 91, + "resultSize" : 2065, + "jvmGcTime" : 5, + "resultSerializationTime" : 1, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 99944, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 21, + "index" : 21, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.015GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 5, + "executorRunTime" : 88, + "resultSize" : 2010, + "jvmGcTime" : 5, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 100270, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 16, + "index" : 16, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.001GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 10, + "executorRunTime" : 84, + "resultSize" : 2010, + "jvmGcTime" : 5, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 108320, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 19, + "index" : 19, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.012GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 5, + "executorRunTime" : 84, + "resultSize" : 2010, + "jvmGcTime" : 5, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 95788, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 9, + "index" : 9, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:06.915GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 9, + "executorRunTime" : 84, + "resultSize" : 2010, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 60489, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 101664, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 20, + "index" : 20, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.014GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 3, + "executorRunTime" : 83, + "resultSize" : 2010, + "jvmGcTime" : 5, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 97716, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 14, + "index" : 14, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:06.925GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 6, + "executorRunTime" : 83, + "resultSize" : 2010, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 95646, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 8, + "index" : 8, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:06.914GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 3, + "executorRunTime" : 80, + "resultSize" : 2010, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 60488, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 121551, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 12, + "index" : 12, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:06.923GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 9, + "executorRunTime" : 77, + "resultSize" : 2010, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 102476, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 13, + "index" : 13, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:06.924GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 9, + "executorRunTime" : 76, + "resultSize" : 2010, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 95004, + "recordsWritten" : 10 + } + } +} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_short_names___runtime_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_short_names___runtime_expectation.json new file mode 100644 index 0000000000000..5657123a2db15 --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_short_names___runtime_expectation.json @@ -0,0 +1,561 @@ +[ { + "taskId" : 6, + "index" : 6, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:06.505GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 29, + "executorRunTime" : 351, + "resultSize" : 2010, + "jvmGcTime" : 7, + "resultSerializationTime" : 1, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 60488, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 4016617, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 5, + "index" : 5, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:06.505GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 30, + "executorRunTime" : 350, + "resultSize" : 2010, + "jvmGcTime" : 7, + "resultSerializationTime" : 1, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 60488, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 3675510, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 1, + "index" : 1, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:06.502GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 31, + "executorRunTime" : 350, + "resultSize" : 2010, + "jvmGcTime" : 7, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 60488, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 3934399, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 4, + "index" : 4, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:06.504GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 31, + "executorRunTime" : 349, + "resultSize" : 2010, + "jvmGcTime" : 7, + "resultSerializationTime" : 1, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 60488, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 83022, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 7, + "index" : 7, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:06.506GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 31, + "executorRunTime" : 349, + "resultSize" : 2010, + "jvmGcTime" : 7, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 60488, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 2579051, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 3, + "index" : 3, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:06.504GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 31, + "executorRunTime" : 349, + "resultSize" : 2010, + "jvmGcTime" : 7, + "resultSerializationTime" : 2, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 60488, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 1311694, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 0, + "index" : 0, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:06.494GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 32, + "executorRunTime" : 349, + "resultSize" : 2010, + "jvmGcTime" : 7, + "resultSerializationTime" : 1, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 49294, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 3842811, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 2, + "index" : 2, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:06.503GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 32, + "executorRunTime" : 348, + "resultSize" : 2010, + "jvmGcTime" : 7, + "resultSerializationTime" : 2, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 60488, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 89885, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 22, + "index" : 22, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.018GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 5, + "executorRunTime" : 93, + "resultSize" : 2065, + "jvmGcTime" : 5, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 143427, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 18, + "index" : 18, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.010GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 10, + "executorRunTime" : 92, + "resultSize" : 2065, + "jvmGcTime" : 5, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 100836, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 17, + "index" : 17, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.005GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 11, + "executorRunTime" : 91, + "resultSize" : 2065, + "jvmGcTime" : 5, + "resultSerializationTime" : 1, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 99944, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 21, + "index" : 21, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.015GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 5, + "executorRunTime" : 88, + "resultSize" : 2010, + "jvmGcTime" : 5, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 100270, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 16, + "index" : 16, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.001GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 10, + "executorRunTime" : 84, + "resultSize" : 2010, + "jvmGcTime" : 5, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 108320, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 19, + "index" : 19, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.012GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 5, + "executorRunTime" : 84, + "resultSize" : 2010, + "jvmGcTime" : 5, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 95788, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 9, + "index" : 9, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:06.915GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 9, + "executorRunTime" : 84, + "resultSize" : 2010, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 60489, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 101664, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 20, + "index" : 20, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.014GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 3, + "executorRunTime" : 83, + "resultSize" : 2010, + "jvmGcTime" : 5, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 97716, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 14, + "index" : 14, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:06.925GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 6, + "executorRunTime" : 83, + "resultSize" : 2010, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 95646, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 8, + "index" : 8, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:06.914GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 3, + "executorRunTime" : 80, + "resultSize" : 2010, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 60488, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 121551, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 12, + "index" : 12, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:06.923GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 9, + "executorRunTime" : 77, + "resultSize" : 2010, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 102476, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 13, + "index" : 13, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:06.924GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 9, + "executorRunTime" : 76, + "resultSize" : 2010, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 95004, + "recordsWritten" : 10 + } + } +} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_short_names__runtime_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_short_names__runtime_expectation.json new file mode 100644 index 0000000000000..72fe017e9f85d --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_short_names__runtime_expectation.json @@ -0,0 +1,561 @@ +[ { + "taskId" : 40, + "index" : 40, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.197GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 4, + "executorRunTime" : 14, + "resultSize" : 2065, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 94792, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 86, + "index" : 86, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.374GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 3, + "executorRunTime" : 16, + "resultSize" : 2065, + "jvmGcTime" : 0, + "resultSerializationTime" : 1, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 95848, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 41, + "index" : 41, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.200GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 2, + "executorRunTime" : 16, + "resultSize" : 2065, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 90765, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 68, + "index" : 68, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.306GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 2, + "executorRunTime" : 16, + "resultSize" : 2065, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 101750, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 58, + "index" : 58, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.263GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 3, + "executorRunTime" : 16, + "resultSize" : 2065, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 97521, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 43, + "index" : 43, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.204GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 2, + "executorRunTime" : 16, + "resultSize" : 2065, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 171516, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 57, + "index" : 57, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.257GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 3, + "executorRunTime" : 16, + "resultSize" : 2065, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 96849, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 59, + "index" : 59, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.265GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 3, + "executorRunTime" : 17, + "resultSize" : 2065, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 100753, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 32, + "index" : 32, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.148GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 3, + "executorRunTime" : 17, + "resultSize" : 2065, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 89603, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 87, + "index" : 87, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.374GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 12, + "executorRunTime" : 17, + "resultSize" : 2065, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 102159, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 99, + "index" : 99, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.426GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 2, + "executorRunTime" : 17, + "resultSize" : 2065, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70565, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 133964, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 63, + "index" : 63, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.276GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 20, + "executorRunTime" : 17, + "resultSize" : 2065, + "jvmGcTime" : 5, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 102779, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 90, + "index" : 90, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.385GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 2, + "executorRunTime" : 17, + "resultSize" : 2065, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 98472, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 39, + "index" : 39, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.180GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 2, + "executorRunTime" : 17, + "resultSize" : 2065, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 98748, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 42, + "index" : 42, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.203GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 10, + "executorRunTime" : 17, + "resultSize" : 2065, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 103713, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 51, + "index" : 51, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.242GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 2, + "executorRunTime" : 17, + "resultSize" : 2065, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 96013, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 50, + "index" : 50, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.240GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 4, + "executorRunTime" : 18, + "resultSize" : 2065, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 90836, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 53, + "index" : 53, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.244GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 6, + "executorRunTime" : 18, + "resultSize" : 2065, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 92835, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 44, + "index" : 44, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.205GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 3, + "executorRunTime" : 18, + "resultSize" : 2065, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 98293, + "recordsWritten" : 10 + } + } +}, { + "taskId" : 80, + "index" : 80, + "attempt" : 0, + "launchTime" : "2015-05-06T13:03:07.341GMT", + "executorId" : "driver", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 13, + "executorRunTime" : 18, + "resultSize" : 2065, + "jvmGcTime" : 5, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 70564, + "recordsRead" : 10000 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1710, + "writeTime" : 98069, + "recordsWritten" : 10 + } + } +} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_summary_w__custom_quantiles_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_summary_w__custom_quantiles_expectation.json new file mode 100644 index 0000000000000..bc3c302813de2 --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/stage_task_summary_w__custom_quantiles_expectation.json @@ -0,0 +1,19 @@ +{ + "quantiles" : [ 0.01, 0.5, 0.99 ], + "executorDeserializeTime" : [ 1.0, 3.0, 36.0 ], + "executorRunTime" : [ 16.0, 28.0, 351.0 ], + "resultSize" : [ 2010.0, 2065.0, 2065.0 ], + "jvmGcTime" : [ 0.0, 0.0, 7.0 ], + "resultSerializationTime" : [ 0.0, 0.0, 2.0 ], + "memoryBytesSpilled" : [ 0.0, 0.0, 0.0 ], + "diskBytesSpilled" : [ 0.0, 0.0, 0.0 ], + "inputMetrics" : { + "bytesRead" : [ 60488.0, 70564.0, 70565.0 ], + "recordsRead" : [ 10000.0, 10000.0, 10000.0 ] + }, + "shuffleWriteMetrics" : { + "writeBytes" : [ 1710.0, 1710.0, 1710.0 ], + "writeRecords" : [ 10.0, 10.0, 10.0 ], + "writeTime" : [ 89437.0, 102159.0, 4016617.0 ] + } +} \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_summary_w_shuffle_read_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_summary_w_shuffle_read_expectation.json new file mode 100644 index 0000000000000..e084c839f1d5a --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/stage_task_summary_w_shuffle_read_expectation.json @@ -0,0 +1,19 @@ +{ + "quantiles" : [ 0.05, 0.25, 0.5, 0.75, 0.95 ], + "executorDeserializeTime" : [ 1.0, 2.0, 2.0, 2.0, 3.0 ], + "executorRunTime" : [ 30.0, 74.0, 75.0, 76.0, 79.0 ], + "resultSize" : [ 1034.0, 1034.0, 1034.0, 1034.0, 1034.0 ], + "jvmGcTime" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], + "resultSerializationTime" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], + "memoryBytesSpilled" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], + "diskBytesSpilled" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], + "shuffleReadMetrics" : { + "readBytes" : [ 17100.0, 17100.0, 17100.0, 17100.0, 17100.0 ], + "readRecords" : [ 100.0, 100.0, 100.0, 100.0, 100.0 ], + "remoteBlocksFetched" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], + "localBlocksFetched" : [ 100.0, 100.0, 100.0, 100.0, 100.0 ], + "fetchWaitTime" : [ 0.0, 0.0, 0.0, 1.0, 1.0 ], + "remoteBytesRead" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], + "totalBlocksFetched" : [ 100.0, 100.0, 100.0, 100.0, 100.0 ] + } +} \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_summary_w_shuffle_write_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_summary_w_shuffle_write_expectation.json new file mode 100644 index 0000000000000..6ac7811ce691b --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/stage_task_summary_w_shuffle_write_expectation.json @@ -0,0 +1,19 @@ +{ + "quantiles" : [ 0.05, 0.25, 0.5, 0.75, 0.95 ], + "executorDeserializeTime" : [ 2.0, 2.0, 3.0, 7.0, 31.0 ], + "executorRunTime" : [ 16.0, 18.0, 28.0, 49.0, 349.0 ], + "resultSize" : [ 2010.0, 2065.0, 2065.0, 2065.0, 2065.0 ], + "jvmGcTime" : [ 0.0, 0.0, 0.0, 5.0, 7.0 ], + "resultSerializationTime" : [ 0.0, 0.0, 0.0, 0.0, 1.0 ], + "memoryBytesSpilled" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], + "diskBytesSpilled" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], + "inputMetrics" : { + "bytesRead" : [ 60488.0, 70564.0, 70564.0, 70564.0, 70564.0 ], + "recordsRead" : [ 10000.0, 10000.0, 10000.0, 10000.0, 10000.0 ] + }, + "shuffleWriteMetrics" : { + "writeBytes" : [ 1710.0, 1710.0, 1710.0, 1710.0, 1710.0 ], + "writeRecords" : [ 10.0, 10.0, 10.0, 10.0, 10.0 ], + "writeTime" : [ 90329.0, 95848.0, 102159.0, 121551.0, 2579051.0 ] + } +} \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/stage_with_accumulable_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_with_accumulable_json_expectation.json new file mode 100644 index 0000000000000..32d5731676ad5 --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/stage_with_accumulable_json_expectation.json @@ -0,0 +1,242 @@ +{ + "status" : "COMPLETE", + "stageId" : 0, + "attemptId" : 0, + "numActiveTasks" : 0, + "numCompleteTasks" : 8, + "numFailedTasks" : 0, + "executorRunTime" : 120, + "inputBytes" : 0, + "inputRecords" : 0, + "outputBytes" : 0, + "outputRecords" : 0, + "shuffleReadBytes" : 0, + "shuffleReadRecords" : 0, + "shuffleWriteBytes" : 0, + "shuffleWriteRecords" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "name" : "foreach at :15", + "details" : "org.apache.spark.rdd.RDD.foreach(RDD.scala:765)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)", + "schedulingPool" : "default", + "accumulatorUpdates" : [ { + "id" : 1, + "name" : "my counter", + "value" : "5050" + } ], + "tasks" : { + "2" : { + "taskId" : 2, + "index" : 2, + "attempt" : 0, + "launchTime" : "2015-03-16T19:25:36.522GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ { + "id" : 1, + "name" : "my counter", + "update" : "378", + "value" : "378" + } ], + "taskMetrics" : { + "executorDeserializeTime" : 13, + "executorRunTime" : 15, + "resultSize" : 697, + "jvmGcTime" : 0, + "resultSerializationTime" : 2, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0 + } + }, + "5" : { + "taskId" : 5, + "index" : 5, + "attempt" : 0, + "launchTime" : "2015-03-16T19:25:36.523GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ { + "id" : 1, + "name" : "my counter", + "update" : "897", + "value" : "3750" + } ], + "taskMetrics" : { + "executorDeserializeTime" : 12, + "executorRunTime" : 15, + "resultSize" : 697, + "jvmGcTime" : 0, + "resultSerializationTime" : 2, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0 + } + }, + "4" : { + "taskId" : 4, + "index" : 4, + "attempt" : 0, + "launchTime" : "2015-03-16T19:25:36.522GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ { + "id" : 1, + "name" : "my counter", + "update" : "678", + "value" : "2853" + } ], + "taskMetrics" : { + "executorDeserializeTime" : 12, + "executorRunTime" : 15, + "resultSize" : 697, + "jvmGcTime" : 0, + "resultSerializationTime" : 1, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0 + } + }, + "7" : { + "taskId" : 7, + "index" : 7, + "attempt" : 0, + "launchTime" : "2015-03-16T19:25:36.524GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ { + "id" : 1, + "name" : "my counter", + "update" : "1222", + "value" : "4972" + } ], + "taskMetrics" : { + "executorDeserializeTime" : 12, + "executorRunTime" : 15, + "resultSize" : 697, + "jvmGcTime" : 0, + "resultSerializationTime" : 2, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0 + } + }, + "1" : { + "taskId" : 1, + "index" : 1, + "attempt" : 0, + "launchTime" : "2015-03-16T19:25:36.521GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ { + "id" : 1, + "name" : "my counter", + "update" : "247", + "value" : "2175" + } ], + "taskMetrics" : { + "executorDeserializeTime" : 14, + "executorRunTime" : 15, + "resultSize" : 697, + "jvmGcTime" : 0, + "resultSerializationTime" : 2, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0 + } + }, + "3" : { + "taskId" : 3, + "index" : 3, + "attempt" : 0, + "launchTime" : "2015-03-16T19:25:36.522GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ { + "id" : 1, + "name" : "my counter", + "update" : "572", + "value" : "950" + } ], + "taskMetrics" : { + "executorDeserializeTime" : 13, + "executorRunTime" : 15, + "resultSize" : 697, + "jvmGcTime" : 0, + "resultSerializationTime" : 2, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0 + } + }, + "6" : { + "taskId" : 6, + "index" : 6, + "attempt" : 0, + "launchTime" : "2015-03-16T19:25:36.523GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ { + "id" : 1, + "name" : "my counter", + "update" : "978", + "value" : "1928" + } ], + "taskMetrics" : { + "executorDeserializeTime" : 12, + "executorRunTime" : 15, + "resultSize" : 697, + "jvmGcTime" : 0, + "resultSerializationTime" : 2, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0 + } + }, + "0" : { + "taskId" : 0, + "index" : 0, + "attempt" : 0, + "launchTime" : "2015-03-16T19:25:36.515GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ { + "id" : 1, + "name" : "my counter", + "update" : "78", + "value" : "5050" + } ], + "taskMetrics" : { + "executorDeserializeTime" : 14, + "executorRunTime" : 15, + "resultSize" : 697, + "jvmGcTime" : 0, + "resultSerializationTime" : 2, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0 + } + } + }, + "executorSummary" : { + "" : { + "taskTime" : 418, + "failedTasks" : 0, + "succeededTasks" : 8, + "inputBytes" : 0, + "outputBytes" : 0, + "shuffleRead" : 0, + "shuffleWrite" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0 + } + } +} \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/succeeded_failed_job_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/succeeded_failed_job_list_json_expectation.json new file mode 100644 index 0000000000000..cab4750270dfa --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/succeeded_failed_job_list_json_expectation.json @@ -0,0 +1,43 @@ +[ { + "jobId" : 2, + "name" : "count at :17", + "stageIds" : [ 3 ], + "status" : "SUCCEEDED", + "numTasks" : 8, + "numActiveTasks" : 0, + "numCompletedTasks" : 8, + "numSkippedTasks" : 8, + "numFailedTasks" : 0, + "numActiveStages" : 0, + "numCompletedStages" : 1, + "numSkippedStages" : 0, + "numFailedStages" : 0 +}, { + "jobId" : 1, + "name" : "count at :20", + "stageIds" : [ 1, 2 ], + "status" : "FAILED", + "numTasks" : 16, + "numActiveTasks" : 0, + "numCompletedTasks" : 15, + "numSkippedTasks" : 15, + "numFailedTasks" : 1, + "numActiveStages" : 0, + "numCompletedStages" : 1, + "numSkippedStages" : 0, + "numFailedStages" : 1 +}, { + "jobId" : 0, + "name" : "count at :15", + "stageIds" : [ 0 ], + "status" : "SUCCEEDED", + "numTasks" : 8, + "numActiveTasks" : 0, + "numCompletedTasks" : 8, + "numSkippedTasks" : 8, + "numFailedTasks" : 0, + "numActiveStages" : 0, + "numCompletedStages" : 1, + "numSkippedStages" : 0, + "numFailedStages" : 0 +} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/succeeded_job_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/succeeded_job_list_json_expectation.json new file mode 100644 index 0000000000000..6fd25befbf7e8 --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/succeeded_job_list_json_expectation.json @@ -0,0 +1,29 @@ +[ { + "jobId" : 2, + "name" : "count at :17", + "stageIds" : [ 3 ], + "status" : "SUCCEEDED", + "numTasks" : 8, + "numActiveTasks" : 0, + "numCompletedTasks" : 8, + "numSkippedTasks" : 8, + "numFailedTasks" : 0, + "numActiveStages" : 0, + "numCompletedStages" : 1, + "numSkippedStages" : 0, + "numFailedStages" : 0 +}, { + "jobId" : 0, + "name" : "count at :15", + "stageIds" : [ 0 ], + "status" : "SUCCEEDED", + "numTasks" : 8, + "numActiveTasks" : 0, + "numCompletedTasks" : 8, + "numSkippedTasks" : 8, + "numFailedTasks" : 0, + "numActiveStages" : 0, + "numCompletedStages" : 1, + "numSkippedStages" : 0, + "numFailedStages" : 0 +} ] \ No newline at end of file diff --git a/core/src/test/resources/spark-events/local-1422981759269/APPLICATION_COMPLETE b/core/src/test/resources/spark-events/local-1422981759269/APPLICATION_COMPLETE new file mode 100755 index 0000000000000..e69de29bb2d1d diff --git a/core/src/test/resources/spark-events/local-1422981759269/EVENT_LOG_1 b/core/src/test/resources/spark-events/local-1422981759269/EVENT_LOG_1 new file mode 100755 index 0000000000000..4794e56d1107a --- /dev/null +++ b/core/src/test/resources/spark-events/local-1422981759269/EVENT_LOG_1 @@ -0,0 +1,88 @@ +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"","Host":"localhost","Port":57967},"Maximum Memory":278302556,"Timestamp":1422981759407} +{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre","Java Version":"1.7.0_67 (Oracle Corporation)","Scala Version":"version 2.10.4"},"Spark Properties":{"spark.driver.host":"192.168.1.103","spark.eventLog.enabled":"true","spark.driver.port":"57965","spark.repl.class.uri":"http://192.168.1.103:57964","spark.jars":"","spark.app.name":"Spark shell","spark.scheduler.mode":"FIFO","spark.executor.id":"driver","spark.master":"local[*]","spark.fileserver.uri":"http://192.168.1.103:57966","spark.tachyonStore.folderName":"spark-fd6c823a-8a18-4113-8306-1fa7bb623a7f","spark.app.id":"local-1422981759269"},"System Properties":{"java.io.tmpdir":"/var/folders/36/m29jw1z95qv4ywb1c4n0rz000000gp/T/","line.separator":"\n","path.separator":":","sun.management.compiler":"HotSpot 64-Bit Tiered Compilers","SPARK_SUBMIT":"true","sun.cpu.endian":"little","java.specification.version":"1.7","java.vm.specification.name":"Java Virtual Machine Specification","java.vendor":"Oracle Corporation","java.vm.specification.version":"1.7","user.home":"/Users/irashid","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","ftp.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","sun.arch.data.model":"64","sun.boot.library.path":"/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib","user.dir":"/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4","java.library.path":"/Users/irashid/Library/Java/Extensions:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java:.","sun.cpu.isalist":"","os.arch":"x86_64","java.vm.version":"24.65-b04","java.endorsed.dirs":"/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/endorsed","java.runtime.version":"1.7.0_67-b01","java.vm.info":"mixed mode","java.ext.dirs":"/Users/irashid/Library/Java/Extensions:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/ext:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java","java.runtime.name":"Java(TM) SE Runtime Environment","file.separator":"/","java.class.version":"51.0","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/resources.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/rt.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/sunrsasign.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/jsse.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/jce.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/charsets.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/jfr.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/classes","file.encoding":"UTF-8","user.timezone":"America/Chicago","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"10.9.5","sun.os.patch.level":"unknown","gopherProxySet":"false","java.vm.specification.vendor":"Oracle Corporation","user.country":"US","sun.jnu.encoding":"UTF-8","http.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","user.language":"en","socksNonProxyHosts":"local|*.local|169.254/16|*.169.254/16","java.vendor.url":"http://java.oracle.com/","java.awt.printerjob":"sun.lwawt.macosx.CPrinterJob","java.awt.graphicsenv":"sun.awt.CGraphicsEnvironment","awt.toolkit":"sun.lwawt.macosx.LWCToolkit","os.name":"Mac OS X","java.vm.vendor":"Oracle Corporation","java.vendor.url.bug":"http://bugreport.sun.com/bugreport/","user.name":"irashid","java.vm.name":"Java HotSpot(TM) 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --class org.apache.spark.repl.Main --conf spark.eventLog.enabled=true spark-shell","java.home":"/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre","java.version":"1.7.0_67","sun.io.unicode.encoding":"UnicodeBig"},"Classpath Entries":{"/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/spark-assembly-1.2.0-hadoop2.4.0.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-api-jdo-3.2.6.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-rdbms-3.2.9.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-core-3.2.10.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/conf":"System Classpath"}} +{"Event":"SparkListenerApplicationStart","App Name":"Spark shell","App ID":"local-1422981759269","Timestamp":1422981758277,"User":"irashid"} +{"Event":"SparkListenerJobStart","Job ID":0,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"count at :15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}],"Stage IDs":[0]} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"count at :15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1422981762075,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1422981762081,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1422981762081,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1422981762082,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1422981762083,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1422981762084,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1422981762084,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1422981762085,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1422981762084,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981762632,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":521,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_6","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1422981762081,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981762633,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":520,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_2","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1422981762082,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981762634,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":521,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_3","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1422981762084,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981762634,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":522,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_5","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1422981762083,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981762635,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":522,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_4","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1422981762075,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981762636,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":11,"Executor Run Time":522,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1422981762085,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981762636,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":9,"Executor Run Time":521,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_7","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1422981762081,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981762637,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":11,"Executor Run Time":522,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_1","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"count at :15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":8,"Memory Size":28000128,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1422981762069,"Completion Time":1422981762637,"Accumulables":[]}} +{"Event":"SparkListenerJobEnd","Job ID":0,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"SparkListenerJobStart","Job ID":1,"Stage Infos":[{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"map at :14","Number of Tasks":8,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.(:14)\n$line10.$read$$iwC$$iwC$$iwC.(:19)\n$line10.$read$$iwC$$iwC.(:21)\n$line10.$read$$iwC.(:23)\n$line10.$read.(:25)\n$line10.$read$.(:29)\n$line10.$read$.()\n$line10.$eval$.(:7)\n$line10.$eval$.()\n$line10.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]},{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"first at :17","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.first(RDD.scala:1093)\n$line11.$read$$iwC$$iwC$$iwC$$iwC.(:17)\n$line11.$read$$iwC$$iwC$$iwC.(:22)\n$line11.$read$$iwC$$iwC.(:24)\n$line11.$read$$iwC.(:26)\n$line11.$read.(:28)\n$line11.$read$.(:32)\n$line11.$read$.()\n$line11.$eval$.(:7)\n$line11.$eval$.()\n$line11.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}],"Stage IDs":[1,2]} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"map at :14","Number of Tasks":8,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.(:14)\n$line10.$read$$iwC$$iwC$$iwC.(:19)\n$line10.$read$$iwC$$iwC.(:21)\n$line10.$read$$iwC.(:23)\n$line10.$read.(:25)\n$line10.$read$.(:29)\n$line10.$read$.()\n$line10.$eval$.(:7)\n$line10.$eval$.()\n$line10.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":8,"Index":0,"Attempt":0,"Launch Time":1422981763578,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":9,"Index":1,"Attempt":0,"Launch Time":1422981763578,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":10,"Index":2,"Attempt":0,"Launch Time":1422981763579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":11,"Index":3,"Attempt":0,"Launch Time":1422981763579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":12,"Index":4,"Attempt":0,"Launch Time":1422981763580,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":13,"Index":5,"Attempt":0,"Launch Time":1422981763580,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":14,"Index":6,"Attempt":0,"Launch Time":1422981763581,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":15,"Index":7,"Attempt":0,"Launch Time":1422981763581,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":10,"Index":2,"Attempt":0,"Launch Time":1422981763579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764001,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":406,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":138000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":9,"Index":1,"Attempt":0,"Launch Time":1422981763578,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764002,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":407,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":106000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":12,"Index":4,"Attempt":0,"Launch Time":1422981763580,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764002,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":407,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1645,"Shuffle Write Time":99000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":8,"Index":0,"Attempt":0,"Launch Time":1422981763578,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764003,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":407,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":123000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":15,"Index":7,"Attempt":0,"Launch Time":1422981763581,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764003,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":406,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":108000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":11,"Index":3,"Attempt":0,"Launch Time":1422981763579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764004,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":407,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1647,"Shuffle Write Time":97000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":14,"Index":6,"Attempt":0,"Launch Time":1422981763581,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764004,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":407,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":132000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":13,"Index":5,"Attempt":0,"Launch Time":1422981763580,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764005,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":407,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":81000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"map at :14","Number of Tasks":8,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.(:14)\n$line10.$read$$iwC$$iwC$$iwC.(:19)\n$line10.$read$$iwC$$iwC.(:21)\n$line10.$read$$iwC.(:23)\n$line10.$read.(:25)\n$line10.$read$.(:29)\n$line10.$read$.()\n$line10.$eval$.(:7)\n$line10.$eval$.()\n$line10.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1422981763578,"Completion Time":1422981764005,"Accumulables":[]}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"first at :17","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.first(RDD.scala:1093)\n$line11.$read$$iwC$$iwC$$iwC$$iwC.(:17)\n$line11.$read$$iwC$$iwC$$iwC.(:22)\n$line11.$read$$iwC$$iwC.(:24)\n$line11.$read$$iwC.(:26)\n$line11.$read.(:28)\n$line11.$read$.(:32)\n$line11.$read$.()\n$line11.$eval$.(:7)\n$line11.$eval$.()\n$line11.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":16,"Index":0,"Attempt":0,"Launch Time":1422981764014,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":16,"Index":0,"Attempt":0,"Launch Time":1422981764014,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764045,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":28,"Result Size":1013,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"first at :17","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.first(RDD.scala:1093)\n$line11.$read$$iwC$$iwC$$iwC$$iwC.(:17)\n$line11.$read$$iwC$$iwC$$iwC.(:22)\n$line11.$read$$iwC$$iwC.(:24)\n$line11.$read$$iwC.(:26)\n$line11.$read.(:28)\n$line11.$read$.(:32)\n$line11.$read$.()\n$line11.$eval$.(:7)\n$line11.$eval$.()\n$line11.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1422981764014,"Completion Time":1422981764045,"Accumulables":[]}} +{"Event":"SparkListenerJobEnd","Job ID":1,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"SparkListenerJobStart","Job ID":2,"Stage Infos":[{"Stage ID":3,"Stage Attempt ID":0,"Stage Name":"map at :14","Number of Tasks":8,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.(:14)\n$line10.$read$$iwC$$iwC$$iwC.(:19)\n$line10.$read$$iwC$$iwC.(:21)\n$line10.$read$$iwC.(:23)\n$line10.$read.(:25)\n$line10.$read$.(:29)\n$line10.$read$.()\n$line10.$eval$.(:7)\n$line10.$eval$.()\n$line10.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]},{"Stage ID":4,"Stage Attempt ID":0,"Stage Name":"saveAsTextFile at :19","Number of Tasks":8,"RDD Info":[{"RDD ID":3,"Name":"3","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.saveAsTextFile(RDD.scala:1164)\n$line13.$read$$iwC$$iwC$$iwC$$iwC.(:19)\n$line13.$read$$iwC$$iwC$$iwC.(:24)\n$line13.$read$$iwC$$iwC.(:26)\n$line13.$read$$iwC.(:28)\n$line13.$read.(:30)\n$line13.$read$.(:34)\n$line13.$read$.()\n$line13.$eval$.(:7)\n$line13.$eval$.()\n$line13.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}],"Stage IDs":[3,4]} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":4,"Stage Attempt ID":0,"Stage Name":"saveAsTextFile at :19","Number of Tasks":8,"RDD Info":[{"RDD ID":3,"Name":"3","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.saveAsTextFile(RDD.scala:1164)\n$line13.$read$$iwC$$iwC$$iwC$$iwC.(:19)\n$line13.$read$$iwC$$iwC$$iwC.(:24)\n$line13.$read$$iwC$$iwC.(:26)\n$line13.$read$$iwC.(:28)\n$line13.$read.(:30)\n$line13.$read$.(:34)\n$line13.$read$.()\n$line13.$eval$.(:7)\n$line13.$eval$.()\n$line13.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":17,"Index":0,"Attempt":0,"Launch Time":1422981764396,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":18,"Index":1,"Attempt":0,"Launch Time":1422981764396,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":19,"Index":2,"Attempt":0,"Launch Time":1422981764397,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":20,"Index":3,"Attempt":0,"Launch Time":1422981764397,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":21,"Index":4,"Attempt":0,"Launch Time":1422981764398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":22,"Index":5,"Attempt":0,"Launch Time":1422981764398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":23,"Index":6,"Attempt":0,"Launch Time":1422981764398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":24,"Index":7,"Attempt":0,"Launch Time":1422981764399,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":24,"Index":7,"Attempt":0,"Launch Time":1422981764399,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764642,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":240,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":19,"Index":2,"Attempt":0,"Launch Time":1422981764397,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764643,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":241,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":21,"Index":4,"Attempt":0,"Launch Time":1422981764398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764643,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":240,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":22,"Index":5,"Attempt":0,"Launch Time":1422981764398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764645,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":241,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":23,"Index":6,"Attempt":0,"Launch Time":1422981764398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764645,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":241,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":17,"Index":0,"Attempt":0,"Launch Time":1422981764396,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764646,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":243,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":18,"Index":1,"Attempt":0,"Launch Time":1422981764396,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764646,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":243,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":20,"Index":3,"Attempt":0,"Launch Time":1422981764397,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764648,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":247,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":4,"Stage Attempt ID":0,"Stage Name":"saveAsTextFile at :19","Number of Tasks":8,"RDD Info":[{"RDD ID":3,"Name":"3","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.saveAsTextFile(RDD.scala:1164)\n$line13.$read$$iwC$$iwC$$iwC$$iwC.(:19)\n$line13.$read$$iwC$$iwC$$iwC.(:24)\n$line13.$read$$iwC$$iwC.(:26)\n$line13.$read$$iwC.(:28)\n$line13.$read.(:30)\n$line13.$read$.(:34)\n$line13.$read$.()\n$line13.$eval$.(:7)\n$line13.$eval$.()\n$line13.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1422981764396,"Completion Time":1422981764648,"Accumulables":[]}} +{"Event":"SparkListenerJobEnd","Job ID":2,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"SparkListenerJobStart","Job ID":3,"Stage Infos":[{"Stage ID":5,"Stage Attempt ID":0,"Stage Name":"count at :17","Number of Tasks":8,"RDD Info":[{"RDD ID":5,"Name":"/Users/irashid/spark-examples/tmp_data/sums","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":4,"Name":"/Users/irashid/spark-examples/tmp_data/sums","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.(:17)\n$line15.$read$$iwC$$iwC$$iwC.(:22)\n$line15.$read$$iwC$$iwC.(:24)\n$line15.$read$$iwC.(:26)\n$line15.$read.(:28)\n$line15.$read$.(:32)\n$line15.$read$.()\n$line15.$eval$.(:7)\n$line15.$eval$.()\n$line15.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}],"Stage IDs":[5]} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":5,"Stage Attempt ID":0,"Stage Name":"count at :17","Number of Tasks":8,"RDD Info":[{"RDD ID":5,"Name":"/Users/irashid/spark-examples/tmp_data/sums","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":4,"Name":"/Users/irashid/spark-examples/tmp_data/sums","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.(:17)\n$line15.$read$$iwC$$iwC$$iwC.(:22)\n$line15.$read$$iwC$$iwC.(:24)\n$line15.$read$$iwC.(:26)\n$line15.$read.(:28)\n$line15.$read$.(:32)\n$line15.$read$.()\n$line15.$eval$.(:7)\n$line15.$eval$.()\n$line15.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":25,"Index":0,"Attempt":0,"Launch Time":1422981765026,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":26,"Index":1,"Attempt":0,"Launch Time":1422981765026,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":27,"Index":2,"Attempt":0,"Launch Time":1422981765027,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":28,"Index":3,"Attempt":0,"Launch Time":1422981765027,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":29,"Index":4,"Attempt":0,"Launch Time":1422981765027,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":30,"Index":5,"Attempt":0,"Launch Time":1422981765028,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":31,"Index":6,"Attempt":0,"Launch Time":1422981765028,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":32,"Index":7,"Attempt":0,"Launch Time":1422981765029,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":27,"Index":2,"Attempt":0,"Launch Time":1422981765027,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981765045,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":30,"Index":5,"Attempt":0,"Launch Time":1422981765028,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981765046,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":25,"Index":0,"Attempt":0,"Launch Time":1422981765026,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981765046,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":12,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":32}}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":29,"Index":4,"Attempt":0,"Launch Time":1422981765027,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981765047,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":28,"Index":3,"Attempt":0,"Launch Time":1422981765027,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981765047,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":12,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":32,"Index":7,"Attempt":0,"Launch Time":1422981765029,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981765048,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":31,"Index":6,"Attempt":0,"Launch Time":1422981765028,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981765048,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":14,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":26,"Index":1,"Attempt":0,"Launch Time":1422981765026,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981765049,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":12,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":32}}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":5,"Stage Attempt ID":0,"Stage Name":"count at :17","Number of Tasks":8,"RDD Info":[{"RDD ID":5,"Name":"/Users/irashid/spark-examples/tmp_data/sums","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":4,"Name":"/Users/irashid/spark-examples/tmp_data/sums","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.(:17)\n$line15.$read$$iwC$$iwC$$iwC.(:22)\n$line15.$read$$iwC$$iwC.(:24)\n$line15.$read$$iwC.(:26)\n$line15.$read.(:28)\n$line15.$read$.(:32)\n$line15.$read$.()\n$line15.$eval$.(:7)\n$line15.$eval$.()\n$line15.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1422981765026,"Completion Time":1422981765050,"Accumulables":[]}} +{"Event":"SparkListenerJobEnd","Job ID":3,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"SparkListenerApplicationEnd","Timestamp":1422981766912} diff --git a/core/src/test/resources/spark-events/local-1422981759269/SPARK_VERSION_1.2.0 b/core/src/test/resources/spark-events/local-1422981759269/SPARK_VERSION_1.2.0 new file mode 100755 index 0000000000000..e69de29bb2d1d diff --git a/core/src/test/resources/spark-events/local-1422981780767/APPLICATION_COMPLETE b/core/src/test/resources/spark-events/local-1422981780767/APPLICATION_COMPLETE new file mode 100755 index 0000000000000..e69de29bb2d1d diff --git a/core/src/test/resources/spark-events/local-1422981780767/EVENT_LOG_1 b/core/src/test/resources/spark-events/local-1422981780767/EVENT_LOG_1 new file mode 100755 index 0000000000000..f14a000bf2c28 --- /dev/null +++ b/core/src/test/resources/spark-events/local-1422981780767/EVENT_LOG_1 @@ -0,0 +1,82 @@ +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"","Host":"localhost","Port":57971},"Maximum Memory":278302556,"Timestamp":1422981780906} +{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre","Java Version":"1.7.0_67 (Oracle Corporation)","Scala Version":"version 2.10.4"},"Spark Properties":{"spark.driver.host":"192.168.1.103","spark.eventLog.enabled":"true","spark.driver.port":"57969","spark.repl.class.uri":"http://192.168.1.103:57968","spark.jars":"","spark.app.name":"Spark shell","spark.scheduler.mode":"FIFO","spark.executor.id":"driver","spark.master":"local[*]","spark.fileserver.uri":"http://192.168.1.103:57970","spark.tachyonStore.folderName":"spark-3f19daee-844c-41d0-a3fc-5e3e508f9731","spark.app.id":"local-1422981780767"},"System Properties":{"java.io.tmpdir":"/var/folders/36/m29jw1z95qv4ywb1c4n0rz000000gp/T/","line.separator":"\n","path.separator":":","sun.management.compiler":"HotSpot 64-Bit Tiered Compilers","SPARK_SUBMIT":"true","sun.cpu.endian":"little","java.specification.version":"1.7","java.vm.specification.name":"Java Virtual Machine Specification","java.vendor":"Oracle Corporation","java.vm.specification.version":"1.7","user.home":"/Users/irashid","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","ftp.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","sun.arch.data.model":"64","sun.boot.library.path":"/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib","user.dir":"/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4","java.library.path":"/Users/irashid/Library/Java/Extensions:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java:.","sun.cpu.isalist":"","os.arch":"x86_64","java.vm.version":"24.65-b04","java.endorsed.dirs":"/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/endorsed","java.runtime.version":"1.7.0_67-b01","java.vm.info":"mixed mode","java.ext.dirs":"/Users/irashid/Library/Java/Extensions:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/ext:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java","java.runtime.name":"Java(TM) SE Runtime Environment","file.separator":"/","java.class.version":"51.0","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/resources.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/rt.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/sunrsasign.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/jsse.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/jce.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/charsets.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/jfr.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/classes","file.encoding":"UTF-8","user.timezone":"America/Chicago","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"10.9.5","sun.os.patch.level":"unknown","gopherProxySet":"false","java.vm.specification.vendor":"Oracle Corporation","user.country":"US","sun.jnu.encoding":"UTF-8","http.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","user.language":"en","socksNonProxyHosts":"local|*.local|169.254/16|*.169.254/16","java.vendor.url":"http://java.oracle.com/","java.awt.printerjob":"sun.lwawt.macosx.CPrinterJob","java.awt.graphicsenv":"sun.awt.CGraphicsEnvironment","awt.toolkit":"sun.lwawt.macosx.LWCToolkit","os.name":"Mac OS X","java.vm.vendor":"Oracle Corporation","java.vendor.url.bug":"http://bugreport.sun.com/bugreport/","user.name":"irashid","java.vm.name":"Java HotSpot(TM) 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --class org.apache.spark.repl.Main --conf spark.eventLog.enabled=true spark-shell","java.home":"/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre","java.version":"1.7.0_67","sun.io.unicode.encoding":"UnicodeBig"},"Classpath Entries":{"/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/spark-assembly-1.2.0-hadoop2.4.0.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-api-jdo-3.2.6.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-rdbms-3.2.9.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-core-3.2.10.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/conf":"System Classpath"}} +{"Event":"SparkListenerApplicationStart","App Name":"Spark shell","App ID":"local-1422981780767","Timestamp":1422981779720,"User":"irashid"} +{"Event":"SparkListenerJobStart","Job ID":0,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"count at :15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}],"Stage IDs":[0]} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"count at :15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1422981784234,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1422981784240,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1422981784240,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1422981784241,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1422981784241,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1422981784242,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1422981784242,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1422981784243,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1422981784241,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981784812,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":11,"Executor Run Time":543,"Result Size":1268,"JVM GC Time":25,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_3","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1422981784240,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981784814,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":542,"Result Size":1268,"JVM GC Time":25,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_1","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1422981784234,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981784816,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":542,"Result Size":1268,"JVM GC Time":25,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1422981784243,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981784816,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":11,"Executor Run Time":543,"Result Size":1268,"JVM GC Time":25,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_7","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1422981784242,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981784817,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":541,"Result Size":1268,"JVM GC Time":25,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_5","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1422981784241,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981784817,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":11,"Executor Run Time":542,"Result Size":1268,"JVM GC Time":25,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_4","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1422981784242,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981784818,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":11,"Executor Run Time":543,"Result Size":1268,"JVM GC Time":25,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_6","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1422981784240,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981784818,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":11,"Executor Run Time":542,"Result Size":1268,"JVM GC Time":25,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_2","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"count at :15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":8,"Memory Size":28000128,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1422981784228,"Completion Time":1422981784819,"Accumulables":[]}} +{"Event":"SparkListenerJobEnd","Job ID":0,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"SparkListenerJobStart","Job ID":1,"Stage Infos":[{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"map at :14","Number of Tasks":8,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.(:14)\n$line10.$read$$iwC$$iwC$$iwC.(:19)\n$line10.$read$$iwC$$iwC.(:21)\n$line10.$read$$iwC.(:23)\n$line10.$read.(:25)\n$line10.$read$.(:29)\n$line10.$read$.()\n$line10.$eval$.(:7)\n$line10.$eval$.()\n$line10.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]},{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"count at :20","Number of Tasks":8,"RDD Info":[{"RDD ID":3,"Name":"3","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line11.$read$$iwC$$iwC$$iwC$$iwC.(:20)\n$line11.$read$$iwC$$iwC$$iwC.(:25)\n$line11.$read$$iwC$$iwC.(:27)\n$line11.$read$$iwC.(:29)\n$line11.$read.(:31)\n$line11.$read$.(:35)\n$line11.$read$.()\n$line11.$eval$.(:7)\n$line11.$eval$.()\n$line11.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}],"Stage IDs":[1,2]} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"map at :14","Number of Tasks":8,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.(:14)\n$line10.$read$$iwC$$iwC$$iwC.(:19)\n$line10.$read$$iwC$$iwC.(:21)\n$line10.$read$$iwC.(:23)\n$line10.$read.(:25)\n$line10.$read$.(:29)\n$line10.$read$.()\n$line10.$eval$.(:7)\n$line10.$eval$.()\n$line10.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":8,"Index":0,"Attempt":0,"Launch Time":1422981785829,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":9,"Index":1,"Attempt":0,"Launch Time":1422981785830,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":10,"Index":2,"Attempt":0,"Launch Time":1422981785830,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":11,"Index":3,"Attempt":0,"Launch Time":1422981785830,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":12,"Index":4,"Attempt":0,"Launch Time":1422981785831,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":13,"Index":5,"Attempt":0,"Launch Time":1422981785831,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":14,"Index":6,"Attempt":0,"Launch Time":1422981785832,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":15,"Index":7,"Attempt":0,"Launch Time":1422981785833,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":14,"Index":6,"Attempt":0,"Launch Time":1422981785832,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786282,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":434,"Result Size":1902,"JVM GC Time":19,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":88000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":8,"Index":0,"Attempt":0,"Launch Time":1422981785829,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786283,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":435,"Result Size":1902,"JVM GC Time":19,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":94000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":15,"Index":7,"Attempt":0,"Launch Time":1422981785833,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786283,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":435,"Result Size":1902,"JVM GC Time":19,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":79000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":13,"Index":5,"Attempt":0,"Launch Time":1422981785831,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786283,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":434,"Result Size":1902,"JVM GC Time":19,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":73000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":11,"Index":3,"Attempt":0,"Launch Time":1422981785830,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786284,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":434,"Result Size":1902,"JVM GC Time":19,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1647,"Shuffle Write Time":83000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":9,"Index":1,"Attempt":0,"Launch Time":1422981785830,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786284,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":436,"Result Size":1902,"JVM GC Time":19,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":98000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":12,"Index":4,"Attempt":0,"Launch Time":1422981785831,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786285,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":434,"Result Size":1902,"JVM GC Time":19,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1645,"Shuffle Write Time":101000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":10,"Index":2,"Attempt":0,"Launch Time":1422981785830,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786286,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":434,"Result Size":1902,"JVM GC Time":19,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":76000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"map at :14","Number of Tasks":8,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.(:14)\n$line10.$read$$iwC$$iwC$$iwC.(:19)\n$line10.$read$$iwC$$iwC.(:21)\n$line10.$read$$iwC.(:23)\n$line10.$read.(:25)\n$line10.$read$.(:29)\n$line10.$read$.()\n$line10.$eval$.(:7)\n$line10.$eval$.()\n$line10.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1422981785829,"Completion Time":1422981786286,"Accumulables":[]}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"count at :20","Number of Tasks":8,"RDD Info":[{"RDD ID":3,"Name":"3","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line11.$read$$iwC$$iwC$$iwC$$iwC.(:20)\n$line11.$read$$iwC$$iwC$$iwC.(:25)\n$line11.$read$$iwC$$iwC.(:27)\n$line11.$read$$iwC.(:29)\n$line11.$read.(:31)\n$line11.$read$.(:35)\n$line11.$read$.()\n$line11.$eval$.(:7)\n$line11.$eval$.()\n$line11.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":16,"Index":0,"Attempt":0,"Launch Time":1422981786296,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":17,"Index":1,"Attempt":0,"Launch Time":1422981786297,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":18,"Index":2,"Attempt":0,"Launch Time":1422981786297,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":19,"Index":3,"Attempt":0,"Launch Time":1422981786297,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":20,"Index":4,"Attempt":0,"Launch Time":1422981786298,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":21,"Index":5,"Attempt":0,"Launch Time":1422981786298,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":22,"Index":6,"Attempt":0,"Launch Time":1422981786298,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":23,"Index":7,"Attempt":0,"Launch Time":1422981786299,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":20,"Index":4,"Attempt":0,"Launch Time":1422981786298,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786337,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":34,"Result Size":862,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":21,"Index":5,"Attempt":0,"Launch Time":1422981786298,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786339,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":35,"Result Size":862,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":17,"Index":1,"Attempt":0,"Launch Time":1422981786297,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786340,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":35,"Result Size":862,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":1,"Remote Bytes Read":0}}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":18,"Index":2,"Attempt":0,"Launch Time":1422981786297,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786340,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":34,"Result Size":862,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":16,"Index":0,"Attempt":0,"Launch Time":1422981786296,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786340,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":35,"Result Size":862,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":23,"Index":7,"Attempt":0,"Launch Time":1422981786299,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786341,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":35,"Result Size":862,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":22,"Index":6,"Attempt":0,"Launch Time":1422981786298,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786342,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":34,"Result Size":862,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"ExceptionFailure","Class Name":"java.lang.RuntimeException","Description":"got a 3, failing","Stack Trace":[{"Declaring Class":"$line11.$read$$iwC$$iwC$$iwC$$iwC$$anonfun$1","Method Name":"apply","File Name":"","Line Number":18},{"Declaring Class":"$line11.$read$$iwC$$iwC$$iwC$$iwC$$anonfun$1","Method Name":"apply","File Name":"","Line Number":17},{"Declaring Class":"scala.collection.Iterator$$anon$11","Method Name":"next","File Name":"Iterator.scala","Line Number":328},{"Declaring Class":"org.apache.spark.util.Utils$","Method Name":"getIteratorSize","File Name":"Utils.scala","Line Number":1311},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":910},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":910},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$4","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1314},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$4","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1314},{"Declaring Class":"org.apache.spark.scheduler.ResultTask","Method Name":"runTask","File Name":"ResultTask.scala","Line Number":61},{"Declaring Class":"org.apache.spark.scheduler.Task","Method Name":"run","File Name":"Task.scala","Line Number":56},{"Declaring Class":"org.apache.spark.executor.Executor$TaskRunner","Method Name":"run","File Name":"Executor.scala","Line Number":196},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor","Method Name":"runWorker","File Name":"ThreadPoolExecutor.java","Line Number":1145},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor$Worker","Method Name":"run","File Name":"ThreadPoolExecutor.java","Line Number":615},{"Declaring Class":"java.lang.Thread","Method Name":"run","File Name":"Thread.java","Line Number":745}],"Full Stack Trace":"java.lang.RuntimeException: got a 3, failing\n\tat $line11.$read$$iwC$$iwC$$iwC$$iwC$$anonfun$1.apply(:18)\n\tat $line11.$read$$iwC$$iwC$$iwC$$iwC$$anonfun$1.apply(:17)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:328)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1311)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:910)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:910)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$4.apply(SparkContext.scala:1314)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$4.apply(SparkContext.scala:1314)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:61)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:56)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:196)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)\n\tat java.lang.Thread.run(Thread.java:745)\n","Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":36,"Result Size":0,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}},"Task Info":{"Task ID":19,"Index":3,"Attempt":0,"Launch Time":1422981786297,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786343,"Failed":true,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":36,"Result Size":0,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"count at :20","Number of Tasks":8,"RDD Info":[{"RDD ID":3,"Name":"3","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line11.$read$$iwC$$iwC$$iwC$$iwC.(:20)\n$line11.$read$$iwC$$iwC$$iwC.(:25)\n$line11.$read$$iwC$$iwC.(:27)\n$line11.$read$$iwC.(:29)\n$line11.$read.(:31)\n$line11.$read$.(:35)\n$line11.$read$.()\n$line11.$eval$.(:7)\n$line11.$eval$.()\n$line11.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1422981786296,"Completion Time":1422981786347,"Failure Reason":"Job aborted due to stage failure: Task 3 in stage 2.0 failed 1 times, most recent failure: Lost task 3.0 in stage 2.0 (TID 19, localhost): java.lang.RuntimeException: got a 3, failing\n\tat $line11.$read$$iwC$$iwC$$iwC$$iwC$$anonfun$1.apply(:18)\n\tat $line11.$read$$iwC$$iwC$$iwC$$iwC$$anonfun$1.apply(:17)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:328)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1311)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:910)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:910)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$4.apply(SparkContext.scala:1314)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$4.apply(SparkContext.scala:1314)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:61)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:56)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:196)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)\n\tat java.lang.Thread.run(Thread.java:745)\n\nDriver stacktrace:","Accumulables":[]}} +{"Event":"SparkListenerJobEnd","Job ID":1,"Job Result":{"Result":"JobFailed","Exception":{"Message":"Job aborted due to stage failure: Task 3 in stage 2.0 failed 1 times, most recent failure: Lost task 3.0 in stage 2.0 (TID 19, localhost): java.lang.RuntimeException: got a 3, failing\n\tat $line11.$read$$iwC$$iwC$$iwC$$iwC$$anonfun$1.apply(:18)\n\tat $line11.$read$$iwC$$iwC$$iwC$$iwC$$anonfun$1.apply(:17)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:328)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1311)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:910)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:910)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$4.apply(SparkContext.scala:1314)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$4.apply(SparkContext.scala:1314)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:61)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:56)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:196)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)\n\tat java.lang.Thread.run(Thread.java:745)\n\nDriver stacktrace:","Stack Trace":[{"Declaring Class":"org.apache.spark.scheduler.DAGScheduler","Method Name":"org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages","File Name":"DAGScheduler.scala","Line Number":1214},{"Declaring Class":"org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1","Method Name":"apply","File Name":"DAGScheduler.scala","Line Number":1203},{"Declaring Class":"org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1","Method Name":"apply","File Name":"DAGScheduler.scala","Line Number":1202},{"Declaring Class":"scala.collection.mutable.ResizableArray$class","Method Name":"foreach","File Name":"ResizableArray.scala","Line Number":59},{"Declaring Class":"scala.collection.mutable.ArrayBuffer","Method Name":"foreach","File Name":"ArrayBuffer.scala","Line Number":47},{"Declaring Class":"org.apache.spark.scheduler.DAGScheduler","Method Name":"abortStage","File Name":"DAGScheduler.scala","Line Number":1202},{"Declaring Class":"org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1","Method Name":"apply","File Name":"DAGScheduler.scala","Line Number":696},{"Declaring Class":"org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1","Method Name":"apply","File Name":"DAGScheduler.scala","Line Number":696},{"Declaring Class":"scala.Option","Method Name":"foreach","File Name":"Option.scala","Line Number":236},{"Declaring Class":"org.apache.spark.scheduler.DAGScheduler","Method Name":"handleTaskSetFailed","File Name":"DAGScheduler.scala","Line Number":696},{"Declaring Class":"org.apache.spark.scheduler.DAGSchedulerEventProcessActor$$anonfun$receive$2","Method Name":"applyOrElse","File Name":"DAGScheduler.scala","Line Number":1420},{"Declaring Class":"akka.actor.Actor$class","Method Name":"aroundReceive","File Name":"Actor.scala","Line Number":465},{"Declaring Class":"org.apache.spark.scheduler.DAGSchedulerEventProcessActor","Method Name":"aroundReceive","File Name":"DAGScheduler.scala","Line Number":1375},{"Declaring Class":"akka.actor.ActorCell","Method Name":"receiveMessage","File Name":"ActorCell.scala","Line Number":516},{"Declaring Class":"akka.actor.ActorCell","Method Name":"invoke","File Name":"ActorCell.scala","Line Number":487},{"Declaring Class":"akka.dispatch.Mailbox","Method Name":"processMailbox","File Name":"Mailbox.scala","Line Number":238},{"Declaring Class":"akka.dispatch.Mailbox","Method Name":"run","File Name":"Mailbox.scala","Line Number":220},{"Declaring Class":"akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask","Method Name":"exec","File Name":"AbstractDispatcher.scala","Line Number":393},{"Declaring Class":"scala.concurrent.forkjoin.ForkJoinTask","Method Name":"doExec","File Name":"ForkJoinTask.java","Line Number":260},{"Declaring Class":"scala.concurrent.forkjoin.ForkJoinPool$WorkQueue","Method Name":"runTask","File Name":"ForkJoinPool.java","Line Number":1339},{"Declaring Class":"scala.concurrent.forkjoin.ForkJoinPool","Method Name":"runWorker","File Name":"ForkJoinPool.java","Line Number":1979},{"Declaring Class":"scala.concurrent.forkjoin.ForkJoinWorkerThread","Method Name":"run","File Name":"ForkJoinWorkerThread.java","Line Number":107}]}}} +{"Event":"SparkListenerJobStart","Job ID":2,"Stage Infos":[{"Stage ID":3,"Stage Attempt ID":0,"Stage Name":"count at :17","Number of Tasks":8,"RDD Info":[{"RDD ID":6,"Name":"/Users/irashid/spark-examples/tmp_data/sums_2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":5,"Name":"/Users/irashid/spark-examples/tmp_data/sums_2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line19.$read$$iwC$$iwC$$iwC$$iwC.(:17)\n$line19.$read$$iwC$$iwC$$iwC.(:22)\n$line19.$read$$iwC$$iwC.(:24)\n$line19.$read$$iwC.(:26)\n$line19.$read.(:28)\n$line19.$read$.(:32)\n$line19.$read$.()\n$line19.$eval$.(:7)\n$line19.$eval$.()\n$line19.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}],"Stage IDs":[3]} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":3,"Stage Attempt ID":0,"Stage Name":"count at :17","Number of Tasks":8,"RDD Info":[{"RDD ID":6,"Name":"/Users/irashid/spark-examples/tmp_data/sums_2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":5,"Name":"/Users/irashid/spark-examples/tmp_data/sums_2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line19.$read$$iwC$$iwC$$iwC$$iwC.(:17)\n$line19.$read$$iwC$$iwC$$iwC.(:22)\n$line19.$read$$iwC$$iwC.(:24)\n$line19.$read$$iwC.(:26)\n$line19.$read.(:28)\n$line19.$read$.(:32)\n$line19.$read$.()\n$line19.$eval$.(:7)\n$line19.$eval$.()\n$line19.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":3,"Stage Attempt ID":0,"Task Info":{"Task ID":24,"Index":0,"Attempt":0,"Launch Time":1422981787191,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":3,"Stage Attempt ID":0,"Task Info":{"Task ID":25,"Index":1,"Attempt":0,"Launch Time":1422981787191,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":3,"Stage Attempt ID":0,"Task Info":{"Task ID":26,"Index":2,"Attempt":0,"Launch Time":1422981787192,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":3,"Stage Attempt ID":0,"Task Info":{"Task ID":27,"Index":3,"Attempt":0,"Launch Time":1422981787192,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":3,"Stage Attempt ID":0,"Task Info":{"Task ID":28,"Index":4,"Attempt":0,"Launch Time":1422981787193,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":3,"Stage Attempt ID":0,"Task Info":{"Task ID":29,"Index":5,"Attempt":0,"Launch Time":1422981787193,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":3,"Stage Attempt ID":0,"Task Info":{"Task ID":30,"Index":6,"Attempt":0,"Launch Time":1422981787194,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":3,"Stage Attempt ID":0,"Task Info":{"Task ID":31,"Index":7,"Attempt":0,"Launch Time":1422981787194,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":3,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":25,"Index":1,"Attempt":0,"Launch Time":1422981787191,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981787222,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":5,"Executor Run Time":20,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":32}}} +{"Event":"SparkListenerTaskEnd","Stage ID":3,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":24,"Index":0,"Attempt":0,"Launch Time":1422981787191,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981787223,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":5,"Executor Run Time":20,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":32}}} +{"Event":"SparkListenerTaskEnd","Stage ID":3,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":26,"Index":2,"Attempt":0,"Launch Time":1422981787192,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981787223,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":22,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} +{"Event":"SparkListenerTaskEnd","Stage ID":3,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":31,"Index":7,"Attempt":0,"Launch Time":1422981787194,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981787223,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":5,"Executor Run Time":19,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} +{"Event":"SparkListenerTaskEnd","Stage ID":3,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":29,"Index":5,"Attempt":0,"Launch Time":1422981787193,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981787224,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":5,"Executor Run Time":19,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} +{"Event":"SparkListenerTaskEnd","Stage ID":3,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":27,"Index":3,"Attempt":0,"Launch Time":1422981787192,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981787224,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":6,"Executor Run Time":19,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} +{"Event":"SparkListenerTaskEnd","Stage ID":3,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":28,"Index":4,"Attempt":0,"Launch Time":1422981787193,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981787225,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":22,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} +{"Event":"SparkListenerTaskEnd","Stage ID":3,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":30,"Index":6,"Attempt":0,"Launch Time":1422981787194,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981787225,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":21,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":3,"Stage Attempt ID":0,"Stage Name":"count at :17","Number of Tasks":8,"RDD Info":[{"RDD ID":6,"Name":"/Users/irashid/spark-examples/tmp_data/sums_2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":5,"Name":"/Users/irashid/spark-examples/tmp_data/sums_2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line19.$read$$iwC$$iwC$$iwC$$iwC.(:17)\n$line19.$read$$iwC$$iwC$$iwC.(:22)\n$line19.$read$$iwC$$iwC.(:24)\n$line19.$read$$iwC.(:26)\n$line19.$read.(:28)\n$line19.$read$.(:32)\n$line19.$read$.()\n$line19.$eval$.(:7)\n$line19.$eval$.()\n$line19.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1422981787191,"Completion Time":1422981787226,"Accumulables":[]}} +{"Event":"SparkListenerJobEnd","Job ID":2,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"SparkListenerApplicationEnd","Timestamp":1422981788731} diff --git a/core/src/test/resources/spark-events/local-1422981780767/SPARK_VERSION_1.2.0 b/core/src/test/resources/spark-events/local-1422981780767/SPARK_VERSION_1.2.0 new file mode 100755 index 0000000000000..e69de29bb2d1d diff --git a/core/src/test/resources/spark-events/local-1425081759269/APPLICATION_COMPLETE b/core/src/test/resources/spark-events/local-1425081759269/APPLICATION_COMPLETE new file mode 100755 index 0000000000000..e69de29bb2d1d diff --git a/core/src/test/resources/spark-events/local-1425081759269/EVENT_LOG_1 b/core/src/test/resources/spark-events/local-1425081759269/EVENT_LOG_1 new file mode 100755 index 0000000000000..9745b36b09e44 --- /dev/null +++ b/core/src/test/resources/spark-events/local-1425081759269/EVENT_LOG_1 @@ -0,0 +1,88 @@ +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"","Host":"localhost","Port":57967},"Maximum Memory":278302556,"Timestamp":1425081759407} +{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre","Java Version":"1.7.0_67 (Oracle Corporation)","Scala Version":"version 2.10.4"},"Spark Properties":{"spark.driver.host":"192.168.1.103","spark.eventLog.enabled":"true","spark.driver.port":"57965","spark.repl.class.uri":"http://192.168.1.103:57964","spark.jars":"","spark.app.name":"Spark shell","spark.scheduler.mode":"FIFO","spark.executor.id":"driver","spark.master":"local[*]","spark.fileserver.uri":"http://192.168.1.103:57966","spark.tachyonStore.folderName":"spark-fd6c823a-8a18-4113-8306-1fa7bb623a7f","spark.app.id":"local-1425081759269"},"System Properties":{"java.io.tmpdir":"/var/folders/36/m29jw1z95qv4ywb1c4n0rz000000gp/T/","line.separator":"\n","path.separator":":","sun.management.compiler":"HotSpot 64-Bit Tiered Compilers","SPARK_SUBMIT":"true","sun.cpu.endian":"little","java.specification.version":"1.7","java.vm.specification.name":"Java Virtual Machine Specification","java.vendor":"Oracle Corporation","java.vm.specification.version":"1.7","user.home":"/Users/irashid","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","ftp.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","sun.arch.data.model":"64","sun.boot.library.path":"/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib","user.dir":"/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4","java.library.path":"/Users/irashid/Library/Java/Extensions:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java:.","sun.cpu.isalist":"","os.arch":"x86_64","java.vm.version":"24.65-b04","java.endorsed.dirs":"/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/endorsed","java.runtime.version":"1.7.0_67-b01","java.vm.info":"mixed mode","java.ext.dirs":"/Users/irashid/Library/Java/Extensions:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/ext:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java","java.runtime.name":"Java(TM) SE Runtime Environment","file.separator":"/","java.class.version":"51.0","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/resources.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/rt.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/sunrsasign.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/jsse.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/jce.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/charsets.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/jfr.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/classes","file.encoding":"UTF-8","user.timezone":"America/Chicago","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"10.9.5","sun.os.patch.level":"unknown","gopherProxySet":"false","java.vm.specification.vendor":"Oracle Corporation","user.country":"US","sun.jnu.encoding":"UTF-8","http.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","user.language":"en","socksNonProxyHosts":"local|*.local|169.254/16|*.169.254/16","java.vendor.url":"http://java.oracle.com/","java.awt.printerjob":"sun.lwawt.macosx.CPrinterJob","java.awt.graphicsenv":"sun.awt.CGraphicsEnvironment","awt.toolkit":"sun.lwawt.macosx.LWCToolkit","os.name":"Mac OS X","java.vm.vendor":"Oracle Corporation","java.vendor.url.bug":"http://bugreport.sun.com/bugreport/","user.name":"irashid","java.vm.name":"Java HotSpot(TM) 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --class org.apache.spark.repl.Main --conf spark.eventLog.enabled=true spark-shell","java.home":"/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre","java.version":"1.7.0_67","sun.io.unicode.encoding":"UnicodeBig"},"Classpath Entries":{"/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/spark-assembly-1.2.0-hadoop2.4.0.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-api-jdo-3.2.6.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-rdbms-3.2.9.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-core-3.2.10.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/conf":"System Classpath"}} +{"Event":"SparkListenerApplicationStart","App Name":"Spark shell","App ID":"local-1425081759269","Timestamp":1425081758277,"User":"irashid"} +{"Event":"SparkListenerJobStart","Job ID":0,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"count at :15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}],"Stage IDs":[0]} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"count at :15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1425081762075,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1425081762081,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1425081762081,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1425081762082,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1425081762083,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1425081762084,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1425081762084,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1425081762085,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1425081762084,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081762632,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":521,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_6","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1425081762081,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081762633,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":520,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_2","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1425081762082,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081762634,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":521,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_3","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1425081762084,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081762634,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":522,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_5","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1425081762083,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081762635,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":522,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_4","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1425081762075,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081762636,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":11,"Executor Run Time":522,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1425081762085,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081762636,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":9,"Executor Run Time":521,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_7","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1425081762081,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081762637,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":11,"Executor Run Time":522,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_1","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"count at :15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":8,"Memory Size":28000128,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1425081762069,"Completion Time":1425081762637,"Accumulables":[]}} +{"Event":"SparkListenerJobEnd","Job ID":0,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"SparkListenerJobStart","Job ID":1,"Stage Infos":[{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"map at :14","Number of Tasks":8,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.(:14)\n$line10.$read$$iwC$$iwC$$iwC.(:19)\n$line10.$read$$iwC$$iwC.(:21)\n$line10.$read$$iwC.(:23)\n$line10.$read.(:25)\n$line10.$read$.(:29)\n$line10.$read$.()\n$line10.$eval$.(:7)\n$line10.$eval$.()\n$line10.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]},{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"first at :17","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.first(RDD.scala:1093)\n$line11.$read$$iwC$$iwC$$iwC$$iwC.(:17)\n$line11.$read$$iwC$$iwC$$iwC.(:22)\n$line11.$read$$iwC$$iwC.(:24)\n$line11.$read$$iwC.(:26)\n$line11.$read.(:28)\n$line11.$read$.(:32)\n$line11.$read$.()\n$line11.$eval$.(:7)\n$line11.$eval$.()\n$line11.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}],"Stage IDs":[1,2]} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"map at :14","Number of Tasks":8,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.(:14)\n$line10.$read$$iwC$$iwC$$iwC.(:19)\n$line10.$read$$iwC$$iwC.(:21)\n$line10.$read$$iwC.(:23)\n$line10.$read.(:25)\n$line10.$read$.(:29)\n$line10.$read$.()\n$line10.$eval$.(:7)\n$line10.$eval$.()\n$line10.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":8,"Index":0,"Attempt":0,"Launch Time":1425081763578,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":9,"Index":1,"Attempt":0,"Launch Time":1425081763578,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":10,"Index":2,"Attempt":0,"Launch Time":1425081763579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":11,"Index":3,"Attempt":0,"Launch Time":1425081763579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":12,"Index":4,"Attempt":0,"Launch Time":1425081763580,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":13,"Index":5,"Attempt":0,"Launch Time":1425081763580,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":14,"Index":6,"Attempt":0,"Launch Time":1425081763581,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":15,"Index":7,"Attempt":0,"Launch Time":1425081763581,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":10,"Index":2,"Attempt":0,"Launch Time":1425081763579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764001,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":406,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":138000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":9,"Index":1,"Attempt":0,"Launch Time":1425081763578,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764002,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":407,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":106000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":12,"Index":4,"Attempt":0,"Launch Time":1425081763580,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764002,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":407,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1645,"Shuffle Write Time":99000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":8,"Index":0,"Attempt":0,"Launch Time":1425081763578,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764003,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":407,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":123000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":15,"Index":7,"Attempt":0,"Launch Time":1425081763581,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764003,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":406,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":108000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":11,"Index":3,"Attempt":0,"Launch Time":1425081763579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764004,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":407,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1647,"Shuffle Write Time":97000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":14,"Index":6,"Attempt":0,"Launch Time":1425081763581,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764004,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":407,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":132000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":13,"Index":5,"Attempt":0,"Launch Time":1425081763580,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764005,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":407,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":81000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"map at :14","Number of Tasks":8,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.(:14)\n$line10.$read$$iwC$$iwC$$iwC.(:19)\n$line10.$read$$iwC$$iwC.(:21)\n$line10.$read$$iwC.(:23)\n$line10.$read.(:25)\n$line10.$read$.(:29)\n$line10.$read$.()\n$line10.$eval$.(:7)\n$line10.$eval$.()\n$line10.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1425081763578,"Completion Time":1425081764005,"Accumulables":[]}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"first at :17","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.first(RDD.scala:1093)\n$line11.$read$$iwC$$iwC$$iwC$$iwC.(:17)\n$line11.$read$$iwC$$iwC$$iwC.(:22)\n$line11.$read$$iwC$$iwC.(:24)\n$line11.$read$$iwC.(:26)\n$line11.$read.(:28)\n$line11.$read$.(:32)\n$line11.$read$.()\n$line11.$eval$.(:7)\n$line11.$eval$.()\n$line11.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":16,"Index":0,"Attempt":0,"Launch Time":1425081764014,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":16,"Index":0,"Attempt":0,"Launch Time":1425081764014,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764045,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":28,"Result Size":1013,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"first at :17","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.first(RDD.scala:1093)\n$line11.$read$$iwC$$iwC$$iwC$$iwC.(:17)\n$line11.$read$$iwC$$iwC$$iwC.(:22)\n$line11.$read$$iwC$$iwC.(:24)\n$line11.$read$$iwC.(:26)\n$line11.$read.(:28)\n$line11.$read$.(:32)\n$line11.$read$.()\n$line11.$eval$.(:7)\n$line11.$eval$.()\n$line11.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1425081764014,"Completion Time":1425081764045,"Accumulables":[]}} +{"Event":"SparkListenerJobEnd","Job ID":1,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"SparkListenerJobStart","Job ID":2,"Stage Infos":[{"Stage ID":3,"Stage Attempt ID":0,"Stage Name":"map at :14","Number of Tasks":8,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.(:14)\n$line10.$read$$iwC$$iwC$$iwC.(:19)\n$line10.$read$$iwC$$iwC.(:21)\n$line10.$read$$iwC.(:23)\n$line10.$read.(:25)\n$line10.$read$.(:29)\n$line10.$read$.()\n$line10.$eval$.(:7)\n$line10.$eval$.()\n$line10.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]},{"Stage ID":4,"Stage Attempt ID":0,"Stage Name":"saveAsTextFile at :19","Number of Tasks":8,"RDD Info":[{"RDD ID":3,"Name":"3","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.saveAsTextFile(RDD.scala:1164)\n$line13.$read$$iwC$$iwC$$iwC$$iwC.(:19)\n$line13.$read$$iwC$$iwC$$iwC.(:24)\n$line13.$read$$iwC$$iwC.(:26)\n$line13.$read$$iwC.(:28)\n$line13.$read.(:30)\n$line13.$read$.(:34)\n$line13.$read$.()\n$line13.$eval$.(:7)\n$line13.$eval$.()\n$line13.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}],"Stage IDs":[3,4]} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":4,"Stage Attempt ID":0,"Stage Name":"saveAsTextFile at :19","Number of Tasks":8,"RDD Info":[{"RDD ID":3,"Name":"3","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.saveAsTextFile(RDD.scala:1164)\n$line13.$read$$iwC$$iwC$$iwC$$iwC.(:19)\n$line13.$read$$iwC$$iwC$$iwC.(:24)\n$line13.$read$$iwC$$iwC.(:26)\n$line13.$read$$iwC.(:28)\n$line13.$read.(:30)\n$line13.$read$.(:34)\n$line13.$read$.()\n$line13.$eval$.(:7)\n$line13.$eval$.()\n$line13.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":17,"Index":0,"Attempt":0,"Launch Time":1425081764396,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":18,"Index":1,"Attempt":0,"Launch Time":1425081764396,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":19,"Index":2,"Attempt":0,"Launch Time":1425081764397,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":20,"Index":3,"Attempt":0,"Launch Time":1425081764397,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":21,"Index":4,"Attempt":0,"Launch Time":1425081764398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":22,"Index":5,"Attempt":0,"Launch Time":1425081764398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":23,"Index":6,"Attempt":0,"Launch Time":1425081764398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":24,"Index":7,"Attempt":0,"Launch Time":1425081764399,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":24,"Index":7,"Attempt":0,"Launch Time":1425081764399,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764642,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":240,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":19,"Index":2,"Attempt":0,"Launch Time":1425081764397,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764643,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":241,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":21,"Index":4,"Attempt":0,"Launch Time":1425081764398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764643,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":240,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":22,"Index":5,"Attempt":0,"Launch Time":1425081764398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764645,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":241,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":23,"Index":6,"Attempt":0,"Launch Time":1425081764398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764645,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":241,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":17,"Index":0,"Attempt":0,"Launch Time":1425081764396,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764646,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":243,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":18,"Index":1,"Attempt":0,"Launch Time":1425081764396,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764646,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":243,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":20,"Index":3,"Attempt":0,"Launch Time":1425081764397,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764648,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":247,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":4,"Stage Attempt ID":0,"Stage Name":"saveAsTextFile at :19","Number of Tasks":8,"RDD Info":[{"RDD ID":3,"Name":"3","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.saveAsTextFile(RDD.scala:1164)\n$line13.$read$$iwC$$iwC$$iwC$$iwC.(:19)\n$line13.$read$$iwC$$iwC$$iwC.(:24)\n$line13.$read$$iwC$$iwC.(:26)\n$line13.$read$$iwC.(:28)\n$line13.$read.(:30)\n$line13.$read$.(:34)\n$line13.$read$.()\n$line13.$eval$.(:7)\n$line13.$eval$.()\n$line13.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1425081764396,"Completion Time":1425081764648,"Accumulables":[]}} +{"Event":"SparkListenerJobEnd","Job ID":2,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"SparkListenerJobStart","Job ID":3,"Stage Infos":[{"Stage ID":5,"Stage Attempt ID":0,"Stage Name":"count at :17","Number of Tasks":8,"RDD Info":[{"RDD ID":5,"Name":"/Users/irashid/spark-examples/tmp_data/sums","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":4,"Name":"/Users/irashid/spark-examples/tmp_data/sums","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.(:17)\n$line15.$read$$iwC$$iwC$$iwC.(:22)\n$line15.$read$$iwC$$iwC.(:24)\n$line15.$read$$iwC.(:26)\n$line15.$read.(:28)\n$line15.$read$.(:32)\n$line15.$read$.()\n$line15.$eval$.(:7)\n$line15.$eval$.()\n$line15.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}],"Stage IDs":[5]} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":5,"Stage Attempt ID":0,"Stage Name":"count at :17","Number of Tasks":8,"RDD Info":[{"RDD ID":5,"Name":"/Users/irashid/spark-examples/tmp_data/sums","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":4,"Name":"/Users/irashid/spark-examples/tmp_data/sums","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.(:17)\n$line15.$read$$iwC$$iwC$$iwC.(:22)\n$line15.$read$$iwC$$iwC.(:24)\n$line15.$read$$iwC.(:26)\n$line15.$read.(:28)\n$line15.$read$.(:32)\n$line15.$read$.()\n$line15.$eval$.(:7)\n$line15.$eval$.()\n$line15.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":25,"Index":0,"Attempt":0,"Launch Time":1425081765026,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":26,"Index":1,"Attempt":0,"Launch Time":1425081765026,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":27,"Index":2,"Attempt":0,"Launch Time":1425081765027,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":28,"Index":3,"Attempt":0,"Launch Time":1425081765027,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":29,"Index":4,"Attempt":0,"Launch Time":1425081765027,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":30,"Index":5,"Attempt":0,"Launch Time":1425081765028,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":31,"Index":6,"Attempt":0,"Launch Time":1425081765028,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":32,"Index":7,"Attempt":0,"Launch Time":1425081765029,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":27,"Index":2,"Attempt":0,"Launch Time":1425081765027,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081765045,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":30,"Index":5,"Attempt":0,"Launch Time":1425081765028,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081765046,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":25,"Index":0,"Attempt":0,"Launch Time":1425081765026,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081765046,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":12,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":32}}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":29,"Index":4,"Attempt":0,"Launch Time":1425081765027,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081765047,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":28,"Index":3,"Attempt":0,"Launch Time":1425081765027,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081765047,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":12,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":32,"Index":7,"Attempt":0,"Launch Time":1425081765029,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081765048,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":31,"Index":6,"Attempt":0,"Launch Time":1425081765028,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081765048,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":14,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":26,"Index":1,"Attempt":0,"Launch Time":1425081765026,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081765049,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":12,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":32}}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":5,"Stage Attempt ID":0,"Stage Name":"count at :17","Number of Tasks":8,"RDD Info":[{"RDD ID":5,"Name":"/Users/irashid/spark-examples/tmp_data/sums","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":4,"Name":"/Users/irashid/spark-examples/tmp_data/sums","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.(:17)\n$line15.$read$$iwC$$iwC$$iwC.(:22)\n$line15.$read$$iwC$$iwC.(:24)\n$line15.$read$$iwC.(:26)\n$line15.$read.(:28)\n$line15.$read$.(:32)\n$line15.$read$.()\n$line15.$eval$.(:7)\n$line15.$eval$.()\n$line15.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1425081765026,"Completion Time":1425081765050,"Accumulables":[]}} +{"Event":"SparkListenerJobEnd","Job ID":3,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"SparkListenerApplicationEnd","Timestamp":1425081766912} diff --git a/core/src/test/resources/spark-events/local-1425081759269/SPARK_VERSION_1.2.0 b/core/src/test/resources/spark-events/local-1425081759269/SPARK_VERSION_1.2.0 new file mode 100755 index 0000000000000..e69de29bb2d1d diff --git a/core/src/test/resources/spark-events/local-1426533911241/APPLICATION_COMPLETE b/core/src/test/resources/spark-events/local-1426533911241/APPLICATION_COMPLETE new file mode 100755 index 0000000000000..e69de29bb2d1d diff --git a/core/src/test/resources/spark-events/local-1426533911241/EVENT_LOG_1 b/core/src/test/resources/spark-events/local-1426533911241/EVENT_LOG_1 new file mode 100755 index 0000000000000..9ef5bd5d92de5 --- /dev/null +++ b/core/src/test/resources/spark-events/local-1426533911241/EVENT_LOG_1 @@ -0,0 +1,24 @@ +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"","Host":"localhost","Port":58610},"Maximum Memory":278019440,"Timestamp":1426533911361} +{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre","Java Version":"1.8.0_25 (Oracle Corporation)","Scala Version":"version 2.10.4"},"Spark Properties":{"spark.driver.host":"192.168.1.105","spark.eventLog.enabled":"true","spark.driver.port":"58608","spark.repl.class.uri":"http://192.168.1.105:58607","spark.jars":"","spark.app.name":"Spark shell","spark.scheduler.mode":"FIFO","spark.executor.id":"driver","spark.master":"local[*]","spark.fileserver.uri":"http://192.168.1.105:58609","spark.tachyonStore.folderName":"spark-5e9b7f26-8e97-4b43-82d6-25c141530da9","spark.app.id":"local-1426533911241"},"System Properties":{"java.io.tmpdir":"/var/folders/36/m29jw1z95qv4ywb1c4n0rz000000gp/T/","line.separator":"\n","path.separator":":","sun.management.compiler":"HotSpot 64-Bit Tiered Compilers","SPARK_SUBMIT":"true","sun.cpu.endian":"little","java.specification.version":"1.8","java.vm.specification.name":"Java Virtual Machine Specification","java.vendor":"Oracle Corporation","java.vm.specification.version":"1.8","user.home":"/Users/irashid","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","ftp.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","sun.arch.data.model":"64","sun.boot.library.path":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib","user.dir":"/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4","java.library.path":"/Users/irashid/Library/Java/Extensions:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java:.","sun.cpu.isalist":"","os.arch":"x86_64","java.vm.version":"25.25-b02","java.endorsed.dirs":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/endorsed","java.runtime.version":"1.8.0_25-b17","java.vm.info":"mixed mode","java.ext.dirs":"/Users/irashid/Library/Java/Extensions:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/ext:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java","java.runtime.name":"Java(TM) SE Runtime Environment","file.separator":"/","java.class.version":"52.0","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/resources.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/rt.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/sunrsasign.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/jsse.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/jce.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/charsets.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/jfr.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/classes","file.encoding":"UTF-8","user.timezone":"America/Chicago","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"10.9.5","sun.os.patch.level":"unknown","gopherProxySet":"false","java.vm.specification.vendor":"Oracle Corporation","user.country":"US","sun.jnu.encoding":"UTF-8","http.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","user.language":"en","socksNonProxyHosts":"local|*.local|169.254/16|*.169.254/16","java.vendor.url":"http://java.oracle.com/","java.awt.printerjob":"sun.lwawt.macosx.CPrinterJob","java.awt.graphicsenv":"sun.awt.CGraphicsEnvironment","awt.toolkit":"sun.lwawt.macosx.LWCToolkit","os.name":"Mac OS X","java.vm.vendor":"Oracle Corporation","java.vendor.url.bug":"http://bugreport.sun.com/bugreport/","user.name":"irashid","java.vm.name":"Java HotSpot(TM) 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --class org.apache.spark.repl.Main --conf spark.eventLog.enabled=true spark-shell","java.home":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre","java.version":"1.8.0_25","sun.io.unicode.encoding":"UnicodeBig"},"Classpath Entries":{"/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/spark-assembly-1.2.0-hadoop2.4.0.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-api-jdo-3.2.6.jar":"System Classpath","/etc/hadoop":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-rdbms-3.2.9.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-core-3.2.10.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/conf":"System Classpath"}} +{"Event":"SparkListenerApplicationStart","App Name":"Spark shell","App ID":"local-1426533911241","Timestamp":1426533910242,"User":"irashid","App Attempt ID":"1"} +{"Event":"SparkListenerJobStart","Job ID":0,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"foreach at :15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.foreach(RDD.scala:765)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}],"Stage IDs":[0]} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"foreach at :15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.foreach(RDD.scala:765)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1426533936515,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1426533936521,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1426533936522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1426533936522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1426533936522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1426533936523,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1426533936523,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1426533936524,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1426533936522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426533936570,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"378","Value":"378"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":13,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1426533936522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426533936572,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"572","Value":"950"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":13,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1426533936523,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426533936574,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"978","Value":"1928"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1426533936521,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426533936574,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"247","Value":"2175"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":14,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1426533936522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426533936574,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"678","Value":"2853"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1426533936523,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426533936575,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"897","Value":"3750"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1426533936524,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426533936575,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"1222","Value":"4972"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1426533936515,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426533936576,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"78","Value":"5050"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":14,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"foreach at :15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.foreach(RDD.scala:765)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1426533936103,"Completion Time":1426533936579,"Accumulables":[{"ID":1,"Name":"my counter","Value":"5050"}]}} +{"Event":"SparkListenerJobEnd","Job ID":0,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"SparkListenerApplicationEnd","Timestamp":1426533945177} diff --git a/core/src/test/resources/spark-events/local-1426533911241/SPARK_VERSION_1.2.0 b/core/src/test/resources/spark-events/local-1426533911241/SPARK_VERSION_1.2.0 new file mode 100755 index 0000000000000..e69de29bb2d1d diff --git a/core/src/test/resources/spark-events/local-1426633911242/APPLICATION_COMPLETE b/core/src/test/resources/spark-events/local-1426633911242/APPLICATION_COMPLETE new file mode 100755 index 0000000000000..e69de29bb2d1d diff --git a/core/src/test/resources/spark-events/local-1426633911242/EVENT_LOG_1 b/core/src/test/resources/spark-events/local-1426633911242/EVENT_LOG_1 new file mode 100755 index 0000000000000..e7043282107d3 --- /dev/null +++ b/core/src/test/resources/spark-events/local-1426633911242/EVENT_LOG_1 @@ -0,0 +1,24 @@ +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"","Host":"localhost","Port":58610},"Maximum Memory":278019440,"Timestamp":1426633911361} +{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre","Java Version":"1.8.0_25 (Oracle Corporation)","Scala Version":"version 2.10.4"},"Spark Properties":{"spark.driver.host":"192.168.1.105","spark.eventLog.enabled":"true","spark.driver.port":"58608","spark.repl.class.uri":"http://192.168.1.105:58607","spark.jars":"","spark.app.name":"Spark shell","spark.scheduler.mode":"FIFO","spark.executor.id":"driver","spark.master":"local[*]","spark.fileserver.uri":"http://192.168.1.105:58609","spark.tachyonStore.folderName":"spark-5e9b7f26-8e97-4b43-82d6-25c141530da9","spark.app.id":"local-1426633911241"},"System Properties":{"java.io.tmpdir":"/var/folders/36/m29jw1z95qv4ywb1c4n0rz000000gp/T/","line.separator":"\n","path.separator":":","sun.management.compiler":"HotSpot 64-Bit Tiered Compilers","SPARK_SUBMIT":"true","sun.cpu.endian":"little","java.specification.version":"1.8","java.vm.specification.name":"Java Virtual Machine Specification","java.vendor":"Oracle Corporation","java.vm.specification.version":"1.8","user.home":"/Users/irashid","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","ftp.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","sun.arch.data.model":"64","sun.boot.library.path":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib","user.dir":"/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4","java.library.path":"/Users/irashid/Library/Java/Extensions:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java:.","sun.cpu.isalist":"","os.arch":"x86_64","java.vm.version":"25.25-b02","java.endorsed.dirs":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/endorsed","java.runtime.version":"1.8.0_25-b17","java.vm.info":"mixed mode","java.ext.dirs":"/Users/irashid/Library/Java/Extensions:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/ext:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java","java.runtime.name":"Java(TM) SE Runtime Environment","file.separator":"/","java.class.version":"52.0","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/resources.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/rt.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/sunrsasign.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/jsse.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/jce.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/charsets.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/jfr.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/classes","file.encoding":"UTF-8","user.timezone":"America/Chicago","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"10.9.5","sun.os.patch.level":"unknown","gopherProxySet":"false","java.vm.specification.vendor":"Oracle Corporation","user.country":"US","sun.jnu.encoding":"UTF-8","http.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","user.language":"en","socksNonProxyHosts":"local|*.local|169.254/16|*.169.254/16","java.vendor.url":"http://java.oracle.com/","java.awt.printerjob":"sun.lwawt.macosx.CPrinterJob","java.awt.graphicsenv":"sun.awt.CGraphicsEnvironment","awt.toolkit":"sun.lwawt.macosx.LWCToolkit","os.name":"Mac OS X","java.vm.vendor":"Oracle Corporation","java.vendor.url.bug":"http://bugreport.sun.com/bugreport/","user.name":"irashid","java.vm.name":"Java HotSpot(TM) 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --class org.apache.spark.repl.Main --conf spark.eventLog.enabled=true spark-shell","java.home":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre","java.version":"1.8.0_25","sun.io.unicode.encoding":"UnicodeBig"},"Classpath Entries":{"/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/spark-assembly-1.2.0-hadoop2.4.0.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-api-jdo-3.2.6.jar":"System Classpath","/etc/hadoop":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-rdbms-3.2.9.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-core-3.2.10.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/conf":"System Classpath"}} +{"Event":"SparkListenerApplicationStart","App Name":"Spark shell","App ID":"local-1426533911241","Timestamp":1426633910242,"User":"irashid","App Attempt ID":"2"} +{"Event":"SparkListenerJobStart","Job ID":0,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"foreach at :15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.foreach(RDD.scala:765)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}],"Stage IDs":[0]} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"foreach at :15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.foreach(RDD.scala:765)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1426633936515,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1426633936521,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1426633936522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1426633936522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1426633936522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1426633936523,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1426633936523,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1426633936524,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1426633936522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426633936570,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"378","Value":"378"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":13,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1426633936522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426633936572,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"572","Value":"950"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":13,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1426633936523,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426633936574,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"978","Value":"1928"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1426633936521,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426633936574,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"247","Value":"2175"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":14,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1426633936522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426633936574,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"678","Value":"2853"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1426633936523,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426633936575,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"897","Value":"3750"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1426633936524,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426633936575,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"1222","Value":"4972"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1426633936515,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426633936576,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"78","Value":"5050"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":14,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"foreach at :15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.foreach(RDD.scala:765)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1426633936103,"Completion Time":1426633936579,"Accumulables":[{"ID":1,"Name":"my counter","Value":"5050"}]}} +{"Event":"SparkListenerJobEnd","Job ID":0,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"SparkListenerApplicationEnd","Timestamp":1426633945177} diff --git a/core/src/test/resources/spark-events/local-1426633911242/SPARK_VERSION_1.2.0 b/core/src/test/resources/spark-events/local-1426633911242/SPARK_VERSION_1.2.0 new file mode 100755 index 0000000000000..e69de29bb2d1d diff --git a/core/src/test/resources/spark-events/local-1430917381534 b/core/src/test/resources/spark-events/local-1430917381534 new file mode 100755 index 0000000000000..24ac478073e3d --- /dev/null +++ b/core/src/test/resources/spark-events/local-1430917381534 @@ -0,0 +1,231 @@ +{"Event":"SparkListenerLogStart","Spark Version":"1.4.0-SNAPSHOT"} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"driver","Host":"localhost","Port":61103},"Maximum Memory":278019440,"Timestamp":1430917381651} +{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre","Java Version":"1.8.0_25 (Oracle Corporation)","Scala Version":"version 2.10.4"},"Spark Properties":{"spark.driver.host":"192.168.1.102","spark.eventLog.enabled":"true","spark.driver.port":"61101","spark.repl.class.uri":"http://192.168.1.102:61100","spark.jars":"","spark.app.name":"Spark shell","spark.scheduler.mode":"FIFO","spark.executor.id":"driver","spark.master":"local[*]","spark.eventLog.dir":"/Users/irashid/github/kraps/core/src/test/resources/spark-events","spark.fileserver.uri":"http://192.168.1.102:61102","spark.tachyonStore.folderName":"spark-aaaf41b3-d1dd-447f-8951-acf51490758b","spark.app.id":"local-1430917381534"},"System Properties":{"java.io.tmpdir":"/var/folders/36/m29jw1z95qv4ywb1c4n0rz000000gp/T/","line.separator":"\n","path.separator":":","sun.management.compiler":"HotSpot 64-Bit Tiered Compilers","SPARK_SUBMIT":"true","sun.cpu.endian":"little","java.specification.version":"1.8","java.vm.specification.name":"Java Virtual Machine Specification","java.vendor":"Oracle Corporation","java.vm.specification.version":"1.8","user.home":"/Users/irashid","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","ftp.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","sun.arch.data.model":"64","sun.boot.library.path":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib","user.dir":"/Users/irashid/github/spark","java.library.path":"/Users/irashid/Library/Java/Extensions:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java:.","sun.cpu.isalist":"","os.arch":"x86_64","java.vm.version":"25.25-b02","java.endorsed.dirs":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/endorsed","java.runtime.version":"1.8.0_25-b17","java.vm.info":"mixed mode","java.ext.dirs":"/Users/irashid/Library/Java/Extensions:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/ext:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java","java.runtime.name":"Java(TM) SE Runtime Environment","file.separator":"/","java.class.version":"52.0","scala.usejavacp":"true","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/resources.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/rt.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/sunrsasign.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/jsse.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/jce.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/charsets.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/jfr.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/classes","file.encoding":"UTF-8","user.timezone":"America/Chicago","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"10.9.5","sun.os.patch.level":"unknown","gopherProxySet":"false","java.vm.specification.vendor":"Oracle Corporation","user.country":"US","sun.jnu.encoding":"UTF-8","http.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","user.language":"en","socksNonProxyHosts":"local|*.local|169.254/16|*.169.254/16","java.vendor.url":"http://java.oracle.com/","java.awt.printerjob":"sun.lwawt.macosx.CPrinterJob","java.awt.graphicsenv":"sun.awt.CGraphicsEnvironment","awt.toolkit":"sun.lwawt.macosx.LWCToolkit","os.name":"Mac OS X","java.vm.vendor":"Oracle Corporation","java.vendor.url.bug":"http://bugreport.sun.com/bugreport/","user.name":"irashid","java.vm.name":"Java HotSpot(TM) 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --conf spark.eventLog.enabled=true --conf spark.eventLog.dir=/Users/irashid/github/kraps/core/src/test/resources/spark-events --class org.apache.spark.repl.Main spark-shell","java.home":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre","java.version":"1.8.0_25","sun.io.unicode.encoding":"UnicodeBig"},"Classpath Entries":{"/etc/hadoop":"System Classpath","/Users/irashid/github/spark/lib_managed/jars/datanucleus-rdbms-3.2.9.jar":"System Classpath","/Users/irashid/github/spark/conf/":"System Classpath","/Users/irashid/github/spark/assembly/target/scala-2.10/spark-assembly-1.4.0-SNAPSHOT-hadoop2.5.0.jar":"System Classpath","/Users/irashid/github/spark/lib_managed/jars/datanucleus-core-3.2.10.jar":"System Classpath","/Users/irashid/github/spark/lib_managed/jars/datanucleus-api-jdo-3.2.6.jar":"System Classpath"}} +{"Event":"SparkListenerApplicationStart","App Name":"Spark shell","App ID":"local-1430917381534","Timestamp":1430917380893,"User":"irashid"} +{"Event":"SparkListenerJobStart","Job ID":0,"Submission Time":1430917386422,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"map at :26","Number of Tasks":100,"RDD Info":[{"RDD ID":3,"Name":"3","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":100,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":100,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"/Users/irashid/spark-examples/tmp_data/raw_data","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":100,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":1,"Name":"/Users/irashid/spark-examples/tmp_data/raw_data","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":100,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:287)\n$line20.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line20.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:31)\n$line20.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:33)\n$line20.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:35)\n$line20.$read$$iwC$$iwC$$iwC$$iwC.(:37)\n$line20.$read$$iwC$$iwC$$iwC.(:39)\n$line20.$read$$iwC$$iwC.(:41)\n$line20.$read$$iwC.(:43)\n$line20.$read.(:45)\n$line20.$read$.(:49)\n$line20.$read$.()\n$line20.$eval$.(:7)\n$line20.$eval$.()\n$line20.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]},{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"collect at :26","Number of Tasks":10,"RDD Info":[{"RDD ID":4,"Name":"4","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":10,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.collect(RDD.scala:819)\n$line20.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line20.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:31)\n$line20.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:33)\n$line20.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:35)\n$line20.$read$$iwC$$iwC$$iwC$$iwC.(:37)\n$line20.$read$$iwC$$iwC$$iwC.(:39)\n$line20.$read$$iwC$$iwC.(:41)\n$line20.$read$$iwC.(:43)\n$line20.$read.(:45)\n$line20.$read$.(:49)\n$line20.$read$.()\n$line20.$eval$.(:7)\n$line20.$eval$.()\n$line20.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]}],"Stage IDs":[0,1],"Properties":{}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"map at :26","Number of Tasks":100,"RDD Info":[{"RDD ID":3,"Name":"3","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":100,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":100,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"/Users/irashid/spark-examples/tmp_data/raw_data","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":100,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":1,"Name":"/Users/irashid/spark-examples/tmp_data/raw_data","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":100,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:287)\n$line20.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line20.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:31)\n$line20.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:33)\n$line20.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:35)\n$line20.$read$$iwC$$iwC$$iwC$$iwC.(:37)\n$line20.$read$$iwC$$iwC$$iwC.(:39)\n$line20.$read$$iwC$$iwC.(:41)\n$line20.$read$$iwC.(:43)\n$line20.$read.(:45)\n$line20.$read$.(:49)\n$line20.$read$.()\n$line20.$eval$.(:7)\n$line20.$eval$.()\n$line20.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]},"Properties":{}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1430917386494,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1430917386502,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1430917386503,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1430917386504,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1430917386504,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1430917386505,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1430917386505,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1430917386506,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":8,"Index":8,"Attempt":0,"Launch Time":1430917386914,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":9,"Index":9,"Attempt":0,"Launch Time":1430917386915,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":10,"Index":10,"Attempt":0,"Launch Time":1430917386916,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":11,"Index":11,"Attempt":0,"Launch Time":1430917386918,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1430917386505,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917386919,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":30,"Executor Run Time":350,"Result Size":2010,"JVM GC Time":7,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":3675510,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":60488,"Records Read":10000}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1430917386503,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917386922,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":32,"Executor Run Time":348,"Result Size":2010,"JVM GC Time":7,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":89885,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":60488,"Records Read":10000}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1430917386502,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917386923,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":31,"Executor Run Time":350,"Result Size":2010,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":3934399,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":60488,"Records Read":10000}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1430917386504,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917386923,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":31,"Executor Run Time":349,"Result Size":2010,"JVM GC Time":7,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":83022,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":60488,"Records Read":10000}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":12,"Index":12,"Attempt":0,"Launch Time":1430917386923,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387016,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1430917386505,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917386924,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":29,"Executor Run Time":351,"Result Size":2010,"JVM GC Time":7,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":4016617,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":60488,"Records Read":10000}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":13,"Index":13,"Attempt":0,"Launch Time":1430917386924,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387062,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":14,"Index":14,"Attempt":0,"Launch Time":1430917386925,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387019,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1430917386504,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917386927,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":31,"Executor Run Time":349,"Result Size":2010,"JVM GC Time":7,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":1311694,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":60488,"Records Read":10000}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":15,"Index":15,"Attempt":0,"Launch Time":1430917386928,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387011,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1430917386494,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917386929,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":32,"Executor Run Time":349,"Result Size":2010,"JVM GC Time":7,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":3842811,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":49294,"Records Read":10000}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1430917386506,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917386929,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":31,"Executor Run Time":349,"Result Size":2010,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":2579051,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":60488,"Records Read":10000}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":16,"Index":16,"Attempt":0,"Launch Time":1430917387001,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387099,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":8,"Index":8,"Attempt":0,"Launch Time":1430917386914,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387002,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":80,"Result Size":2010,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":121551,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":60488,"Records Read":10000}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":17,"Index":17,"Attempt":0,"Launch Time":1430917387005,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387128,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":11,"Index":11,"Attempt":0,"Launch Time":1430917386918,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387007,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":6,"Executor Run Time":75,"Result Size":2010,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":94507,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":18,"Index":18,"Attempt":0,"Launch Time":1430917387010,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387115,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":15,"Index":15,"Attempt":0,"Launch Time":1430917386928,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387011,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":76,"Result Size":2010,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":602780,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":19,"Index":19,"Attempt":0,"Launch Time":1430917387012,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387106,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":20,"Index":20,"Attempt":0,"Launch Time":1430917387014,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387104,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":21,"Index":21,"Attempt":0,"Launch Time":1430917387015,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387111,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":10,"Index":10,"Attempt":0,"Launch Time":1430917386916,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387015,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":8,"Executor Run Time":73,"Result Size":2010,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":94709,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":9,"Index":9,"Attempt":0,"Launch Time":1430917386915,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387016,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":9,"Executor Run Time":84,"Result Size":2010,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":101664,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":60489,"Records Read":10000}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":12,"Index":12,"Attempt":0,"Launch Time":1430917386923,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387016,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":9,"Executor Run Time":77,"Result Size":2010,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":102476,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":22,"Index":22,"Attempt":0,"Launch Time":1430917387018,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387119,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":14,"Index":14,"Attempt":0,"Launch Time":1430917386925,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387019,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":6,"Executor Run Time":83,"Result Size":2010,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":95646,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":23,"Index":23,"Attempt":0,"Launch Time":1430917387031,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387115,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":13,"Index":13,"Attempt":0,"Launch Time":1430917386924,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387062,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":9,"Executor Run Time":76,"Result Size":2010,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":95004,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":24,"Index":24,"Attempt":0,"Launch Time":1430917387098,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":16,"Index":16,"Attempt":0,"Launch Time":1430917387001,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387099,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":84,"Result Size":2010,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":108320,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":25,"Index":25,"Attempt":0,"Launch Time":1430917387103,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":20,"Index":20,"Attempt":0,"Launch Time":1430917387014,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387104,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":83,"Result Size":2010,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":97716,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":26,"Index":26,"Attempt":0,"Launch Time":1430917387105,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":19,"Index":19,"Attempt":0,"Launch Time":1430917387012,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387106,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":5,"Executor Run Time":84,"Result Size":2010,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":95788,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":27,"Index":27,"Attempt":0,"Launch Time":1430917387110,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":21,"Index":21,"Attempt":0,"Launch Time":1430917387015,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387111,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":5,"Executor Run Time":88,"Result Size":2010,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":100270,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":28,"Index":28,"Attempt":0,"Launch Time":1430917387113,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":29,"Index":29,"Attempt":0,"Launch Time":1430917387114,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":23,"Index":23,"Attempt":0,"Launch Time":1430917387031,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387115,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":65,"Result Size":2010,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":91844,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":18,"Index":18,"Attempt":0,"Launch Time":1430917387010,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387115,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":92,"Result Size":2065,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":100836,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":30,"Index":30,"Attempt":0,"Launch Time":1430917387118,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":22,"Index":22,"Attempt":0,"Launch Time":1430917387018,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387119,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":5,"Executor Run Time":93,"Result Size":2065,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":143427,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":31,"Index":31,"Attempt":0,"Launch Time":1430917387127,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":17,"Index":17,"Attempt":0,"Launch Time":1430917387005,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387128,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":11,"Executor Run Time":91,"Result Size":2065,"JVM GC Time":5,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":99944,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":32,"Index":32,"Attempt":0,"Launch Time":1430917387148,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":33,"Index":33,"Attempt":0,"Launch Time":1430917387149,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":24,"Index":24,"Attempt":0,"Launch Time":1430917387098,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387150,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":43,"Result Size":2065,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":157194,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":27,"Index":27,"Attempt":0,"Launch Time":1430917387110,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387151,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":32,"Result Size":2065,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":102019,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":34,"Index":34,"Attempt":0,"Launch Time":1430917387156,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":26,"Index":26,"Attempt":0,"Launch Time":1430917387105,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387157,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":6,"Executor Run Time":38,"Result Size":2065,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":108213,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":35,"Index":35,"Attempt":0,"Launch Time":1430917387161,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":28,"Index":28,"Attempt":0,"Launch Time":1430917387113,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387162,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":29,"Result Size":2065,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":104299,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":36,"Index":36,"Attempt":0,"Launch Time":1430917387164,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":25,"Index":25,"Attempt":0,"Launch Time":1430917387103,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387164,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":49,"Result Size":2065,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":94134,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":37,"Index":37,"Attempt":0,"Launch Time":1430917387165,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":38,"Index":38,"Attempt":0,"Launch Time":1430917387166,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":29,"Index":29,"Attempt":0,"Launch Time":1430917387114,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387166,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":39,"Result Size":2065,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":114938,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":39,"Index":39,"Attempt":0,"Launch Time":1430917387180,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":30,"Index":30,"Attempt":0,"Launch Time":1430917387118,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387180,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":34,"Result Size":2065,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":119770,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":32,"Index":32,"Attempt":0,"Launch Time":1430917387148,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387181,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":17,"Result Size":2065,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":89603,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":40,"Index":40,"Attempt":0,"Launch Time":1430917387197,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":34,"Index":34,"Attempt":0,"Launch Time":1430917387156,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387198,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":6,"Executor Run Time":27,"Result Size":2065,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":127746,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":41,"Index":41,"Attempt":0,"Launch Time":1430917387200,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":31,"Index":31,"Attempt":0,"Launch Time":1430917387127,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387201,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":36,"Executor Run Time":24,"Result Size":2065,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":92619,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":42,"Index":42,"Attempt":0,"Launch Time":1430917387203,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":36,"Index":36,"Attempt":0,"Launch Time":1430917387164,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387204,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":5,"Executor Run Time":29,"Result Size":2065,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":123855,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":43,"Index":43,"Attempt":0,"Launch Time":1430917387204,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":44,"Index":44,"Attempt":0,"Launch Time":1430917387205,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":45,"Index":45,"Attempt":0,"Launch Time":1430917387206,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":37,"Index":37,"Attempt":0,"Launch Time":1430917387165,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387207,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":32,"Result Size":2065,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":111869,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":33,"Index":33,"Attempt":0,"Launch Time":1430917387149,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387207,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":5,"Executor Run Time":43,"Result Size":2065,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":118329,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":46,"Index":46,"Attempt":0,"Launch Time":1430917387210,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":35,"Index":35,"Attempt":0,"Launch Time":1430917387161,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387211,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":5,"Executor Run Time":35,"Result Size":2065,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":160963,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":47,"Index":47,"Attempt":0,"Launch Time":1430917387212,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":39,"Index":39,"Attempt":0,"Launch Time":1430917387180,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387212,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":17,"Result Size":2065,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":98748,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":38,"Index":38,"Attempt":0,"Launch Time":1430917387166,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387213,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":31,"Result Size":2065,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":131158,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":48,"Index":48,"Attempt":0,"Launch Time":1430917387220,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":40,"Index":40,"Attempt":0,"Launch Time":1430917387197,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387221,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":14,"Result Size":2065,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":94792,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":49,"Index":49,"Attempt":0,"Launch Time":1430917387223,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":41,"Index":41,"Attempt":0,"Launch Time":1430917387200,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387224,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":16,"Result Size":2065,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":90765,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":50,"Index":50,"Attempt":0,"Launch Time":1430917387240,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":44,"Index":44,"Attempt":0,"Launch Time":1430917387205,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387242,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":18,"Result Size":2065,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":98293,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":51,"Index":51,"Attempt":0,"Launch Time":1430917387242,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":52,"Index":52,"Attempt":0,"Launch Time":1430917387243,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":43,"Index":43,"Attempt":0,"Launch Time":1430917387204,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387243,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":16,"Result Size":2065,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":171516,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":45,"Index":45,"Attempt":0,"Launch Time":1430917387206,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387243,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":19,"Result Size":2065,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":92985,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":53,"Index":53,"Attempt":0,"Launch Time":1430917387244,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":54,"Index":54,"Attempt":0,"Launch Time":1430917387244,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":42,"Index":42,"Attempt":0,"Launch Time":1430917387203,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387245,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":17,"Result Size":2065,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":103713,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":47,"Index":47,"Attempt":0,"Launch Time":1430917387212,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387245,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":18,"Result Size":2065,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":103015,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":55,"Index":55,"Attempt":0,"Launch Time":1430917387246,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":56,"Index":56,"Attempt":0,"Launch Time":1430917387249,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":48,"Index":48,"Attempt":0,"Launch Time":1430917387220,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387250,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":24,"Result Size":2065,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":139844,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":46,"Index":46,"Attempt":0,"Launch Time":1430917387210,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387253,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":31,"Result Size":2065,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":113322,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":57,"Index":57,"Attempt":0,"Launch Time":1430917387257,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":49,"Index":49,"Attempt":0,"Launch Time":1430917387223,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387257,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":7,"Executor Run Time":23,"Result Size":2065,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":94984,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":58,"Index":58,"Attempt":0,"Launch Time":1430917387263,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":51,"Index":51,"Attempt":0,"Launch Time":1430917387242,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387263,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":17,"Result Size":2065,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":96013,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":59,"Index":59,"Attempt":0,"Launch Time":1430917387265,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":50,"Index":50,"Attempt":0,"Launch Time":1430917387240,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387266,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":18,"Result Size":2065,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":90836,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":60,"Index":60,"Attempt":0,"Launch Time":1430917387269,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":61,"Index":61,"Attempt":0,"Launch Time":1430917387270,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":52,"Index":52,"Attempt":0,"Launch Time":1430917387243,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387271,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":5,"Executor Run Time":18,"Result Size":2065,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":89664,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":62,"Index":62,"Attempt":0,"Launch Time":1430917387272,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":53,"Index":53,"Attempt":0,"Launch Time":1430917387244,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387273,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":6,"Executor Run Time":18,"Result Size":2065,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":92835,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":63,"Index":63,"Attempt":0,"Launch Time":1430917387276,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":55,"Index":55,"Attempt":0,"Launch Time":1430917387246,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387276,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":21,"Result Size":2065,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":108309,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":64,"Index":64,"Attempt":0,"Launch Time":1430917387277,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":57,"Index":57,"Attempt":0,"Launch Time":1430917387257,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387278,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":16,"Result Size":2065,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":96849,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":65,"Index":65,"Attempt":0,"Launch Time":1430917387279,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":56,"Index":56,"Attempt":0,"Launch Time":1430917387249,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387280,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":5,"Executor Run Time":20,"Result Size":2065,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":90329,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":66,"Index":66,"Attempt":0,"Launch Time":1430917387285,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":58,"Index":58,"Attempt":0,"Launch Time":1430917387263,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387286,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":16,"Result Size":2065,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":97521,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":67,"Index":67,"Attempt":0,"Launch Time":1430917387288,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":59,"Index":59,"Attempt":0,"Launch Time":1430917387265,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387288,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":17,"Result Size":2065,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":100753,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":54,"Index":54,"Attempt":0,"Launch Time":1430917387244,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387303,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":18,"Result Size":2065,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":90506,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":68,"Index":68,"Attempt":0,"Launch Time":1430917387306,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":60,"Index":60,"Attempt":0,"Launch Time":1430917387269,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387307,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":8,"Executor Run Time":28,"Result Size":2065,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":156916,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":69,"Index":69,"Attempt":0,"Launch Time":1430917387308,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":62,"Index":62,"Attempt":0,"Launch Time":1430917387272,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387309,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":32,"Result Size":2065,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":100396,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":70,"Index":70,"Attempt":0,"Launch Time":1430917387311,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":64,"Index":64,"Attempt":0,"Launch Time":1430917387277,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387311,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":25,"Result Size":2065,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":104087,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":71,"Index":71,"Attempt":0,"Launch Time":1430917387313,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":72,"Index":72,"Attempt":0,"Launch Time":1430917387314,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":61,"Index":61,"Attempt":0,"Launch Time":1430917387270,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387315,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":36,"Result Size":2065,"JVM GC Time":5,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":100426,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":73,"Index":73,"Attempt":0,"Launch Time":1430917387316,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":63,"Index":63,"Attempt":0,"Launch Time":1430917387276,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387316,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":20,"Executor Run Time":17,"Result Size":2065,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":102779,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":74,"Index":74,"Attempt":0,"Launch Time":1430917387318,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":66,"Index":66,"Attempt":0,"Launch Time":1430917387285,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387319,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":27,"Result Size":2065,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":100823,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":65,"Index":65,"Attempt":0,"Launch Time":1430917387279,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387321,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":27,"Result Size":2065,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":99123,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":75,"Index":75,"Attempt":0,"Launch Time":1430917387323,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":67,"Index":67,"Attempt":0,"Launch Time":1430917387288,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387323,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":14,"Executor Run Time":19,"Result Size":2065,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":92741,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":76,"Index":76,"Attempt":0,"Launch Time":1430917387327,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":68,"Index":68,"Attempt":0,"Launch Time":1430917387306,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387328,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":16,"Result Size":2065,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":101750,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":77,"Index":77,"Attempt":0,"Launch Time":1430917387329,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":69,"Index":69,"Attempt":0,"Launch Time":1430917387308,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387331,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":18,"Result Size":2065,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":89437,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":78,"Index":78,"Attempt":0,"Launch Time":1430917387338,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":71,"Index":71,"Attempt":0,"Launch Time":1430917387313,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387339,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":22,"Result Size":2065,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":183323,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":79,"Index":79,"Attempt":0,"Launch Time":1430917387339,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":70,"Index":70,"Attempt":0,"Launch Time":1430917387311,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387340,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":24,"Result Size":2065,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":105933,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":80,"Index":80,"Attempt":0,"Launch Time":1430917387341,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":74,"Index":74,"Attempt":0,"Launch Time":1430917387318,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387341,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":18,"Result Size":2065,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":98047,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":81,"Index":81,"Attempt":0,"Launch Time":1430917387342,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":72,"Index":72,"Attempt":0,"Launch Time":1430917387314,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387343,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":22,"Result Size":2065,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":139584,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":82,"Index":82,"Attempt":0,"Launch Time":1430917387349,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":75,"Index":75,"Attempt":0,"Launch Time":1430917387323,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387359,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":20,"Result Size":2065,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":134733,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":83,"Index":83,"Attempt":0,"Launch Time":1430917387360,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":73,"Index":73,"Attempt":0,"Launch Time":1430917387316,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387360,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":16,"Executor Run Time":25,"Result Size":2065,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":197728,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":84,"Index":84,"Attempt":0,"Launch Time":1430917387368,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":77,"Index":77,"Attempt":0,"Launch Time":1430917387329,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387368,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":5,"Executor Run Time":30,"Result Size":2065,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":104879,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":85,"Index":85,"Attempt":0,"Launch Time":1430917387371,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":76,"Index":76,"Attempt":0,"Launch Time":1430917387327,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387371,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":38,"Result Size":2065,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":117198,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":86,"Index":86,"Attempt":0,"Launch Time":1430917387374,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":87,"Index":87,"Attempt":0,"Launch Time":1430917387374,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":79,"Index":79,"Attempt":0,"Launch Time":1430917387339,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387375,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":28,"Result Size":2065,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":760569,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":80,"Index":80,"Attempt":0,"Launch Time":1430917387341,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387375,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":13,"Executor Run Time":18,"Result Size":2065,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":98069,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":88,"Index":88,"Attempt":0,"Launch Time":1430917387376,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":78,"Index":78,"Attempt":0,"Launch Time":1430917387338,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387377,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":31,"Result Size":2065,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":94029,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":89,"Index":89,"Attempt":0,"Launch Time":1430917387384,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":82,"Index":82,"Attempt":0,"Launch Time":1430917387349,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387385,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":23,"Result Size":2065,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":103837,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":90,"Index":90,"Attempt":0,"Launch Time":1430917387385,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":81,"Index":81,"Attempt":0,"Launch Time":1430917387342,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387386,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":38,"Result Size":2065,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":102935,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":91,"Index":91,"Attempt":0,"Launch Time":1430917387394,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":84,"Index":84,"Attempt":0,"Launch Time":1430917387368,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387395,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":22,"Result Size":2065,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":97176,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":92,"Index":92,"Attempt":0,"Launch Time":1430917387400,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":85,"Index":85,"Attempt":0,"Launch Time":1430917387371,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387401,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":19,"Result Size":2065,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":103743,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":93,"Index":93,"Attempt":0,"Launch Time":1430917387402,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":86,"Index":86,"Attempt":0,"Launch Time":1430917387374,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387402,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":16,"Result Size":2065,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":95848,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":94,"Index":94,"Attempt":0,"Launch Time":1430917387404,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":83,"Index":83,"Attempt":0,"Launch Time":1430917387360,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387405,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":37,"Result Size":2065,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":134049,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":95,"Index":95,"Attempt":0,"Launch Time":1430917387406,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":96,"Index":96,"Attempt":0,"Launch Time":1430917387406,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":97,"Index":97,"Attempt":0,"Launch Time":1430917387407,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":88,"Index":88,"Attempt":0,"Launch Time":1430917387376,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387407,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":19,"Result Size":2065,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":94269,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":90,"Index":90,"Attempt":0,"Launch Time":1430917387385,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387408,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":17,"Result Size":2065,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":98472,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":98,"Index":98,"Attempt":0,"Launch Time":1430917387410,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":87,"Index":87,"Attempt":0,"Launch Time":1430917387374,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387410,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":17,"Result Size":2065,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":102159,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":89,"Index":89,"Attempt":0,"Launch Time":1430917387384,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387417,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":18,"Result Size":2065,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":97226,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":99,"Index":99,"Attempt":0,"Launch Time":1430917387426,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":91,"Index":91,"Attempt":0,"Launch Time":1430917387394,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387427,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":8,"Executor Run Time":21,"Result Size":2065,"JVM GC Time":4,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":98301,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":94,"Index":94,"Attempt":0,"Launch Time":1430917387404,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387441,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":5,"Executor Run Time":22,"Result Size":2065,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":102455,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":92,"Index":92,"Attempt":0,"Launch Time":1430917387400,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387442,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":31,"Result Size":2065,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":114766,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":96,"Index":96,"Attempt":0,"Launch Time":1430917387406,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387443,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":26,"Result Size":2065,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":100678,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":93,"Index":93,"Attempt":0,"Launch Time":1430917387402,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387443,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":32,"Result Size":2065,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":95818,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":95,"Index":95,"Attempt":0,"Launch Time":1430917387406,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387444,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":28,"Result Size":2065,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":117803,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":98,"Index":98,"Attempt":0,"Launch Time":1430917387410,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387444,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":7,"Executor Run Time":22,"Result Size":2065,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":105039,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":99,"Index":99,"Attempt":0,"Launch Time":1430917387426,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387448,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":17,"Result Size":2065,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":133964,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70565,"Records Read":10000}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":97,"Index":97,"Attempt":0,"Launch Time":1430917387407,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387450,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":38,"Result Size":2065,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1710,"Shuffle Write Time":131987,"Shuffle Records Written":10},"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":70564,"Records Read":10000}}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"map at :26","Number of Tasks":100,"RDD Info":[{"RDD ID":3,"Name":"3","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":100,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":100,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"/Users/irashid/spark-examples/tmp_data/raw_data","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":100,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":1,"Name":"/Users/irashid/spark-examples/tmp_data/raw_data","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":100,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:287)\n$line20.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line20.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:31)\n$line20.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:33)\n$line20.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:35)\n$line20.$read$$iwC$$iwC$$iwC$$iwC.(:37)\n$line20.$read$$iwC$$iwC$$iwC.(:39)\n$line20.$read$$iwC$$iwC.(:41)\n$line20.$read$$iwC.(:43)\n$line20.$read.(:45)\n$line20.$read$.(:49)\n$line20.$read$.()\n$line20.$eval$.(:7)\n$line20.$eval$.()\n$line20.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Submission Time":1430917386485,"Completion Time":1430917387451,"Accumulables":[]}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"collect at :26","Number of Tasks":10,"RDD Info":[{"RDD ID":4,"Name":"4","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":10,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.collect(RDD.scala:819)\n$line20.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line20.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:31)\n$line20.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:33)\n$line20.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:35)\n$line20.$read$$iwC$$iwC$$iwC$$iwC.(:37)\n$line20.$read$$iwC$$iwC$$iwC.(:39)\n$line20.$read$$iwC$$iwC.(:41)\n$line20.$read$$iwC.(:43)\n$line20.$read.(:45)\n$line20.$read$.(:49)\n$line20.$read$.()\n$line20.$eval$.(:7)\n$line20.$eval$.()\n$line20.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]},"Properties":{}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":100,"Index":0,"Attempt":0,"Launch Time":1430917387462,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":101,"Index":1,"Attempt":0,"Launch Time":1430917387464,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":102,"Index":2,"Attempt":0,"Launch Time":1430917387465,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":103,"Index":3,"Attempt":0,"Launch Time":1430917387465,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":104,"Index":4,"Attempt":0,"Launch Time":1430917387465,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":105,"Index":5,"Attempt":0,"Launch Time":1430917387466,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":106,"Index":6,"Attempt":0,"Launch Time":1430917387466,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":107,"Index":7,"Attempt":0,"Launch Time":1430917387466,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":108,"Index":8,"Attempt":0,"Launch Time":1430917387546,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":103,"Index":3,"Attempt":0,"Launch Time":1430917387465,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387546,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":75,"Result Size":1034,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":100,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":17100,"Total Records Read":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":109,"Index":9,"Attempt":0,"Launch Time":1430917387546,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":105,"Index":5,"Attempt":0,"Launch Time":1430917387466,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387547,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":76,"Result Size":1034,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":100,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":17100,"Total Records Read":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":100,"Index":0,"Attempt":0,"Launch Time":1430917387462,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387548,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":74,"Result Size":1034,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":100,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":17100,"Total Records Read":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":107,"Index":7,"Attempt":0,"Launch Time":1430917387466,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387548,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":76,"Result Size":1034,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":100,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":17100,"Total Records Read":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":101,"Index":1,"Attempt":0,"Launch Time":1430917387464,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387548,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":76,"Result Size":1034,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":100,"Fetch Wait Time":1,"Remote Bytes Read":0,"Local Bytes Read":17100,"Total Records Read":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":106,"Index":6,"Attempt":0,"Launch Time":1430917387466,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387549,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":75,"Result Size":1034,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":100,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":17100,"Total Records Read":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":104,"Index":4,"Attempt":0,"Launch Time":1430917387465,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387549,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":74,"Result Size":1034,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":100,"Fetch Wait Time":1,"Remote Bytes Read":0,"Local Bytes Read":17100,"Total Records Read":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":102,"Index":2,"Attempt":0,"Launch Time":1430917387465,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387550,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":79,"Result Size":1034,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":100,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":17100,"Total Records Read":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":108,"Index":8,"Attempt":0,"Launch Time":1430917387546,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387583,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":31,"Result Size":1034,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":100,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":17100,"Total Records Read":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":109,"Index":9,"Attempt":0,"Launch Time":1430917387546,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1430917387585,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":30,"Result Size":1034,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":100,"Fetch Wait Time":1,"Remote Bytes Read":0,"Local Bytes Read":17100,"Total Records Read":100}}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"collect at :26","Number of Tasks":10,"RDD Info":[{"RDD ID":4,"Name":"4","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":10,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.collect(RDD.scala:819)\n$line20.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line20.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:31)\n$line20.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:33)\n$line20.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:35)\n$line20.$read$$iwC$$iwC$$iwC$$iwC.(:37)\n$line20.$read$$iwC$$iwC$$iwC.(:39)\n$line20.$read$$iwC$$iwC.(:41)\n$line20.$read$$iwC.(:43)\n$line20.$read.(:45)\n$line20.$read$.(:49)\n$line20.$read$.()\n$line20.$eval$.(:7)\n$line20.$eval$.()\n$line20.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Submission Time":1430917387462,"Completion Time":1430917387585,"Accumulables":[]}} +{"Event":"SparkListenerJobEnd","Job ID":0,"Completion Time":1430917387589,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"SparkListenerApplicationEnd","Timestamp":1430917391398} diff --git a/core/src/test/scala/org/apache/spark/JsonTestUtils.scala b/core/src/test/scala/org/apache/spark/JsonTestUtils.scala new file mode 100644 index 0000000000000..ba367cd476146 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/JsonTestUtils.scala @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark + +import org.json4s._ +import org.json4s.jackson.JsonMethods + +trait JsonTestUtils { + def assertValidDataInJson(validateJson: JValue, expectedJson: JValue) { + val Diff(c, a, d) = validateJson.diff(expectedJson) + val validatePretty = JsonMethods.pretty(validateJson) + val expectedPretty = JsonMethods.pretty(expectedJson) + val errorMessage = s"Expected:\n$expectedPretty\nFound:\n$validatePretty" + import org.scalactic.TripleEquals._ + assert(c === JNothing, s"$errorMessage\nChanged:\n${JsonMethods.pretty(c)}") + assert(a === JNothing, s"$errorMessage\nAdded:\n${JsonMethods.pretty(a)}") + assert(d === JNothing, s"$errorMessage\nDeleted:\n${JsonMethods.pretty(d)}") + } + +} diff --git a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala index b58d62567afe1..e04a79284175c 100644 --- a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala @@ -28,9 +28,9 @@ import org.scalatest.FunSuite import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, WorkerStateResponse} import org.apache.spark.deploy.master.{ApplicationInfo, DriverInfo, RecoveryState, WorkerInfo} import org.apache.spark.deploy.worker.{DriverRunner, ExecutorRunner} -import org.apache.spark.{SecurityManager, SparkConf} +import org.apache.spark.{JsonTestUtils, SecurityManager, SparkConf} -class JsonProtocolSuite extends FunSuite { +class JsonProtocolSuite extends FunSuite with JsonTestUtils { test("writeApplicationInfo") { val output = JsonProtocol.writeApplicationInfo(createAppInfo()) @@ -136,16 +136,6 @@ class JsonProtocolSuite extends FunSuite { case e: JsonParseException => fail("Invalid Json detected", e) } } - - def assertValidDataInJson(validateJson: JValue, expectedJson: JValue) { - val Diff(c, a, d) = validateJson diff expectedJson - val validatePretty = JsonMethods.pretty(validateJson) - val expectedPretty = JsonMethods.pretty(expectedJson) - val errorMessage = s"Expected:\n$expectedPretty\nFound:\n$validatePretty" - assert(c === JNothing, s"$errorMessage\nChanged:\n${JsonMethods.pretty(c)}") - assert(a === JNothing, s"$errorMessage\nAdded:\n${JsonMethods.pretty(a)}") - assert(d === JNothing, s"$errorMessage\nDelected:\n${JsonMethods.pretty(d)}") - } } object JsonConstants { 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 71ba9c18257b8..318ab5dbc4804 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 @@ -14,22 +14,164 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.spark.deploy.history -import javax.servlet.http.HttpServletRequest - -import scala.collection.mutable +import java.io.{File, FileInputStream, FileWriter, IOException} +import java.net.{HttpURLConnection, URL} +import javax.servlet.http.{HttpServletRequest, HttpServletResponse} -import org.apache.hadoop.fs.Path -import org.mockito.Mockito.{when} -import org.scalatest.FunSuite -import org.scalatest.Matchers +import org.apache.commons.io.{FileUtils, IOUtils} +import org.mockito.Mockito.when +import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} import org.scalatest.mock.MockitoSugar +import org.apache.spark.{JsonTestUtils, SecurityManager, SparkConf} import org.apache.spark.ui.SparkUI -class HistoryServerSuite extends FunSuite with Matchers with MockitoSugar { +/** + * A collection of tests against the historyserver, including comparing responses from the json + * metrics api to a set of known "golden files". If new endpoints / parameters are added, + * cases should be added to this test suite. The expected outcomes can be genered by running + * the HistoryServerSuite.main. Note that this will blindly generate new expectation files matching + * the current behavior -- the developer must verify that behavior is correct. + * + * Similarly, if the behavior is changed, HistoryServerSuite.main can be run to update the + * expectations. However, in general this should be done with extreme caution, as the metrics + * are considered part of Spark's public api. + */ +class HistoryServerSuite extends FunSuite with BeforeAndAfter with Matchers with MockitoSugar + with JsonTestUtils { + + private val logDir = new File("src/test/resources/spark-events") + private val expRoot = new File("src/test/resources/HistoryServerExpectations/") + + private var provider: FsHistoryProvider = null + private var server: HistoryServer = null + private var port: Int = -1 + + def init(): Unit = { + val conf = new SparkConf() + .set("spark.history.fs.logDirectory", logDir.getAbsolutePath) + .set("spark.history.fs.updateInterval", "0") + .set("spark.testing", "true") + provider = new FsHistoryProvider(conf) + provider.checkForLogs() + val securityManager = new SecurityManager(conf) + + server = new HistoryServer(conf, provider, securityManager, 18080) + server.initialize() + server.bind() + port = server.boundPort + } + + def stop(): Unit = { + server.stop() + } + + before { + init() + } + + after{ + stop() + } + + val cases = Seq( + "application list json" -> "applications", + "completed app list json" -> "applications?status=completed", + "running app list json" -> "applications?status=running", + "minDate app list json" -> "applications?minDate=2015-02-10", + "maxDate app list json" -> "applications?maxDate=2015-02-10", + "maxDate2 app list json" -> "applications?maxDate=2015-02-03T10:42:40.000CST", + "one app json" -> "applications/local-1422981780767", + "one app multi-attempt json" -> "applications/local-1426533911241", + "job list json" -> "applications/local-1422981780767/jobs", + "job list from multi-attempt app json(1)" -> "applications/local-1426533911241/1/jobs", + "job list from multi-attempt app json(2)" -> "applications/local-1426533911241/2/jobs", + "one job json" -> "applications/local-1422981780767/jobs/0", + "succeeded job list json" -> "applications/local-1422981780767/jobs?status=succeeded", + "succeeded&failed job list json" -> + "applications/local-1422981780767/jobs?status=succeeded&status=failed", + "executor list json" -> "applications/local-1422981780767/executors", + "stage list json" -> "applications/local-1422981780767/stages", + "complete stage list json" -> "applications/local-1422981780767/stages?status=complete", + "failed stage list json" -> "applications/local-1422981780767/stages?status=failed", + "one stage json" -> "applications/local-1422981780767/stages/1", + "one stage attempt json" -> "applications/local-1422981780767/stages/1/0", + + "stage task summary w shuffle write" + -> "applications/local-1430917381534/stages/0/0/taskSummary", + "stage task summary w shuffle read" + -> "applications/local-1430917381534/stages/1/0/taskSummary", + "stage task summary w/ custom quantiles" -> + "applications/local-1430917381534/stages/0/0/taskSummary?quantiles=0.01,0.5,0.99", + + "stage task list" -> "applications/local-1430917381534/stages/0/0/taskList", + "stage task list w/ offset & length" -> + "applications/local-1430917381534/stages/0/0/taskList?offset=10&length=50", + "stage task list w/ sortBy" -> + "applications/local-1430917381534/stages/0/0/taskList?sortBy=DECREASING_RUNTIME", + "stage task list w/ sortBy short names: -runtime" -> + "applications/local-1430917381534/stages/0/0/taskList?sortBy=-runtime", + "stage task list w/ sortBy short names: runtime" -> + "applications/local-1430917381534/stages/0/0/taskList?sortBy=runtime", + + "stage list with accumulable json" -> "applications/local-1426533911241/1/stages", + "stage with accumulable json" -> "applications/local-1426533911241/1/stages/0/0", + "stage task list from multi-attempt app json(1)" -> + "applications/local-1426533911241/1/stages/0/0/taskList", + "stage task list from multi-attempt app json(2)" -> + "applications/local-1426533911241/2/stages/0/0/taskList", + + "rdd list storage json" -> "applications/local-1422981780767/storage/rdd", + "one rdd storage json" -> "applications/local-1422981780767/storage/rdd/0" + ) + + // run a bunch of characterization tests -- just verify the behavior is the same as what is saved + // in the test resource folder + cases.foreach { case (name, path) => + test(name) { + val (code, jsonOpt, errOpt) = getContentAndCode(path) + code should be (HttpServletResponse.SC_OK) + jsonOpt should be ('defined) + errOpt should be (None) + val json = jsonOpt.get + val exp = IOUtils.toString(new FileInputStream( + new File(expRoot, HistoryServerSuite.sanitizePath(name) + "_expectation.json"))) + // compare the ASTs so formatting differences don't cause failures + import org.json4s._ + import org.json4s.jackson.JsonMethods._ + val jsonAst = parse(json) + val expAst = parse(exp) + assertValidDataInJson(jsonAst, expAst) + } + } + + test("response codes on bad paths") { + val badAppId = getContentAndCode("applications/foobar") + badAppId._1 should be (HttpServletResponse.SC_NOT_FOUND) + badAppId._3 should be (Some("unknown app: foobar")) + + val badStageId = getContentAndCode("applications/local-1422981780767/stages/12345") + badStageId._1 should be (HttpServletResponse.SC_NOT_FOUND) + badStageId._3 should be (Some("unknown stage: 12345")) + + val badStageAttemptId = getContentAndCode("applications/local-1422981780767/stages/1/1") + badStageAttemptId._1 should be (HttpServletResponse.SC_NOT_FOUND) + badStageAttemptId._3 should be (Some("unknown attempt for stage 1. Found attempts: [0]")) + + val badStageId2 = getContentAndCode("applications/local-1422981780767/stages/flimflam") + badStageId2._1 should be (HttpServletResponse.SC_NOT_FOUND) + // will take some mucking w/ jersey to get a better error msg in this case + + val badQuantiles = getContentAndCode( + "applications/local-1430917381534/stages/0/0/taskSummary?quantiles=foo,0.1") + badQuantiles._1 should be (HttpServletResponse.SC_BAD_REQUEST) + badQuantiles._3 should be (Some("Bad value for parameter \"quantiles\". Expected a double, " + + "got \"foo\"")) + + getContentAndCode("foobar")._1 should be (HttpServletResponse.SC_NOT_FOUND) + } test("generate history page with relative links") { val historyServer = mock[HistoryServer] @@ -54,4 +196,75 @@ class HistoryServerSuite extends FunSuite with Matchers with MockitoSugar { } yield (attrs.toString) justHrefs should contain(link) } + + def getContentAndCode(path: String, port: Int = port): (Int, Option[String], Option[String]) = { + HistoryServerSuite.getContentAndCode(new URL(s"http://localhost:$port/json/v1/$path")) + } + + def getUrl(path: String): String = { + HistoryServerSuite.getUrl(new URL(s"http://localhost:$port/json/v1/$path")) + } + + def generateExpectation(name: String, path: String): Unit = { + val json = getUrl(path) + val file = new File(expRoot, HistoryServerSuite.sanitizePath(name) + "_expectation.json") + val out = new FileWriter(file) + out.write(json) + out.close() + } +} + +object HistoryServerSuite { + def main(args: Array[String]): Unit = { + // generate the "expected" results for the characterization tests. Just blindly assume the + // current behavior is correct, and write out the returned json to the test/resource files + + val suite = new HistoryServerSuite + FileUtils.deleteDirectory(suite.expRoot) + suite.expRoot.mkdirs() + try { + suite.init() + suite.cases.foreach { case (name, path) => + suite.generateExpectation(name, path) + } + } finally { + suite.stop() + } + } + + def getContentAndCode(url: URL): (Int, Option[String], Option[String]) = { + val connection = url.openConnection().asInstanceOf[HttpURLConnection] + connection.setRequestMethod("GET") + connection.connect() + val code = connection.getResponseCode() + val inString = try { + val in = Option(connection.getInputStream()) + in.map(IOUtils.toString) + } catch { + case io: IOException => None + } + val errString = try { + val err = Option(connection.getErrorStream()) + err.map(IOUtils.toString) + } catch { + case io: IOException => None + } + (code, inString, errString) + } + + + def sanitizePath(path: String): String = { + // this doesn't need to be perfect, just good enough to avoid collisions + path.replaceAll("\\W", "_") + } + + def getUrl(path: URL): String = { + val (code, resultOpt, error) = getContentAndCode(path) + if (code == 200) { + resultOpt.get + } else { + throw new RuntimeException( + "got code: " + code + " when getting " + path + " w/ error: " + error) + } + } } diff --git a/core/src/test/scala/org/apache/spark/status/api/v1/SimpleDateParamTest.scala b/core/src/test/scala/org/apache/spark/status/api/v1/SimpleDateParamTest.scala new file mode 100644 index 0000000000000..5274df904d395 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/status/api/v1/SimpleDateParamTest.scala @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.status.api.v1 + +import org.scalatest.{Matchers, FunSuite} + +class SimpleDateParamTest extends FunSuite with Matchers { + + test("date parsing") { + new SimpleDateParam("2015-02-20T23:21:17.190GMT").timestamp should be (1424474477190L) + new SimpleDateParam("2015-02-20T17:21:17.190CST").timestamp should be (1424474477190L) + new SimpleDateParam("2015-02-20").timestamp should be (1424390400000L) // GMT + } + +} diff --git a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala index d53d7f3ba5ae7..117b2c3960820 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala @@ -18,11 +18,13 @@ package org.apache.spark.ui import java.net.{HttpURLConnection, URL} -import javax.servlet.http.HttpServletRequest +import javax.servlet.http.{HttpServletResponse, HttpServletRequest} import scala.collection.JavaConversions._ import scala.xml.Node +import org.json4s._ +import org.json4s.jackson.JsonMethods import org.openqa.selenium.htmlunit.HtmlUnitDriver import org.openqa.selenium.{By, WebDriver} import org.scalatest._ @@ -33,8 +35,9 @@ import org.scalatest.time.SpanSugar._ import org.apache.spark.LocalSparkContext._ import org.apache.spark._ import org.apache.spark.api.java.StorageLevels +import org.apache.spark.deploy.history.HistoryServerSuite import org.apache.spark.shuffle.FetchFailedException - +import org.apache.spark.status.api.v1.{JacksonMessageWriter, StageStatus} /** * Selenium tests for the Spark Web UI. @@ -42,6 +45,8 @@ import org.apache.spark.shuffle.FetchFailedException class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with BeforeAndAfterAll { implicit var webDriver: WebDriver = _ + implicit val formats = DefaultFormats + override def beforeAll(): Unit = { webDriver = new HtmlUnitDriver @@ -76,28 +81,42 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before val rdd = sc.parallelize(Seq(1, 2, 3)) rdd.persist(StorageLevels.DISK_ONLY).count() eventually(timeout(5 seconds), interval(50 milliseconds)) { - go to (ui.appUIAddress.stripSuffix("/") + "/storage") + goToUi(ui, "/storage") val tableRowText = findAll(cssSelector("#storage-by-rdd-table td")).map(_.text).toSeq tableRowText should contain (StorageLevels.DISK_ONLY.description) } eventually(timeout(5 seconds), interval(50 milliseconds)) { - go to (ui.appUIAddress.stripSuffix("/") + "/storage/rdd/?id=0") + goToUi(ui, "/storage/rdd/?id=0") val tableRowText = findAll(cssSelector("#rdd-storage-by-block-table td")).map(_.text).toSeq tableRowText should contain (StorageLevels.DISK_ONLY.description) } + val storageJson = getJson(ui, "storage/rdd") + storageJson.children.length should be (1) + (storageJson \ "storageLevel").extract[String] should be (StorageLevels.DISK_ONLY.description) + val rddJson = getJson(ui, "storage/rdd/0") + (rddJson \ "storageLevel").extract[String] should be (StorageLevels.DISK_ONLY.description) + rdd.unpersist() rdd.persist(StorageLevels.MEMORY_ONLY).count() eventually(timeout(5 seconds), interval(50 milliseconds)) { - go to (ui.appUIAddress.stripSuffix("/") + "/storage") + goToUi(ui, "/storage") val tableRowText = findAll(cssSelector("#storage-by-rdd-table td")).map(_.text).toSeq tableRowText should contain (StorageLevels.MEMORY_ONLY.description) } eventually(timeout(5 seconds), interval(50 milliseconds)) { - go to (ui.appUIAddress.stripSuffix("/") + "/storage/rdd/?id=0") + goToUi(ui, "/storage/rdd/?id=0") val tableRowText = findAll(cssSelector("#rdd-storage-by-block-table td")).map(_.text).toSeq tableRowText should contain (StorageLevels.MEMORY_ONLY.description) } + + val updatedStorageJson = getJson(ui, "storage/rdd") + updatedStorageJson.children.length should be (1) + (updatedStorageJson \ "storageLevel").extract[String] should be ( + StorageLevels.MEMORY_ONLY.description) + val updatedRddJson = getJson(ui, "storage/rdd/0") + (updatedRddJson \ "storageLevel").extract[String] should be ( + StorageLevels.MEMORY_ONLY.description) } } @@ -108,10 +127,13 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before sc.parallelize(1 to 10).map { x => throw new Exception()}.collect() } eventually(timeout(5 seconds), interval(50 milliseconds)) { - go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/stages") + goToUi(sc, "/stages") find(id("active")) should be(None) // Since we hide empty tables find(id("failed")).get.text should be("Failed Stages (1)") } + val stageJson = getJson(sc.ui.get, "stages") + stageJson.children.length should be (1) + (stageJson \ "status").extract[String] should be (StageStatus.FAILED.name()) // Regression test for SPARK-2105 class NotSerializable @@ -120,12 +142,15 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before sc.parallelize(1 to 10).map { x => unserializableObject}.collect() } eventually(timeout(5 seconds), interval(50 milliseconds)) { - go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/stages") + goToUi(sc, "/stages") find(id("active")) should be(None) // Since we hide empty tables // The failure occurs before the stage becomes active, hence we should still show only one // failed stage, not two: find(id("failed")).get.text should be("Failed Stages (1)") } + + val updatedStageJson = getJson(sc.ui.get, "stages") + updatedStageJson should be (stageJson) } } @@ -138,7 +163,7 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before withSpark(newSparkContext(killEnabled = true)) { sc => runSlowJob(sc) eventually(timeout(5 seconds), interval(50 milliseconds)) { - go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/stages") + goToUi(sc, "/stages") assert(hasKillLink) } } @@ -146,7 +171,7 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before withSpark(newSparkContext(killEnabled = false)) { sc => runSlowJob(sc) eventually(timeout(5 seconds), interval(50 milliseconds)) { - go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/stages") + goToUi(sc, "/stages") assert(!hasKillLink) } } @@ -157,7 +182,7 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before // If no job has been run in a job group, then "(Job Group)" should not appear in the header sc.parallelize(Seq(1, 2, 3)).count() eventually(timeout(5 seconds), interval(50 milliseconds)) { - go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/jobs") + goToUi(sc, "/jobs") val tableHeaders = findAll(cssSelector("th")).map(_.text).toSeq tableHeaders should not contain "Job Id (Job Group)" } @@ -165,10 +190,22 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before sc.setJobGroup("my-job-group", "my-job-group-description") sc.parallelize(Seq(1, 2, 3)).count() eventually(timeout(5 seconds), interval(50 milliseconds)) { - go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/jobs") + goToUi(sc, "/jobs") val tableHeaders = findAll(cssSelector("th")).map(_.text).toSeq tableHeaders should contain ("Job Id (Job Group)") } + + val jobJson = getJson(sc.ui.get, "jobs") + for { + job @ JObject(_) <- jobJson + JInt(jobId) <- job \ "jobId" + jobGroup = job \ "jobGroup" + } { + jobId.toInt match { + case 0 => jobGroup should be (JNothing) + case 1 => jobGroup should be (JString("my-job-group")) + } + } } } @@ -195,7 +232,7 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before } mappedData.count() eventually(timeout(5 seconds), interval(50 milliseconds)) { - go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/jobs") + goToUi(sc, "/jobs") find(cssSelector(".stage-progress-cell")).get.text should be ("2/2 (1 failed)") // Ideally, the following test would pass, but currently we overcount completed tasks // if task recomputations occur: @@ -204,6 +241,32 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before // of completed tasks may be higher: find(cssSelector(".progress-cell .progress")).get.text should be ("3/2 (1 failed)") } + val jobJson = getJson(sc.ui.get, "jobs") + (jobJson \ "numTasks").extract[Int]should be (2) + (jobJson \ "numCompletedTasks").extract[Int] should be (3) + (jobJson \ "numFailedTasks").extract[Int] should be (1) + (jobJson \ "numCompletedStages").extract[Int] should be (2) + (jobJson \ "numFailedStages").extract[Int] should be (1) + val stageJson = getJson(sc.ui.get, "stages") + + for { + stage @ JObject(_) <- stageJson + JString(status) <- stage \ "status" + JInt(stageId) <- stage \ "stageId" + JInt(attemptId) <- stage \ "attemptId" + } { + val exp = if (attemptId == 0 && stageId == 1) StageStatus.FAILED else StageStatus.COMPLETE + status should be (exp.name()) + } + + for { + stageId <- 0 to 1 + attemptId <- 0 to 1 + } { + val exp = if (attemptId == 0 && stageId == 1) StageStatus.FAILED else StageStatus.COMPLETE + val stageJson = getJson(sc.ui.get, s"stages/$stageId/$attemptId") + (stageJson \ "status").extract[String] should be (exp.name()) + } } } @@ -218,7 +281,7 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before // Start the job: rdd.countAsync() eventually(timeout(10 seconds), interval(50 milliseconds)) { - go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/jobs/job/?id=0") + goToUi(sc, "/jobs/job/?id=0") find(id("active")).get.text should be ("Active Stages (1)") find(id("pending")).get.text should be ("Pending Stages (2)") // Essentially, we want to check that none of the stage rows show @@ -244,7 +307,7 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before rdd.count() rdd.count() eventually(timeout(10 seconds), interval(50 milliseconds)) { - go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/jobs") + goToUi(sc, "/jobs") // The completed jobs table should have two rows. The first row will be the most recent job: val firstRow = find(cssSelector("tbody tr")).get.underlying val firstRowColumns = firstRow.findElements(By.tagName("td")) @@ -271,7 +334,7 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before rdd.count() rdd.count() eventually(timeout(10 seconds), interval(50 milliseconds)) { - go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/jobs/job/?id=1") + goToUi(sc, "/jobs/job/?id=1") find(id("pending")) should be (None) find(id("active")) should be (None) find(id("failed")) should be (None) @@ -299,7 +362,7 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before rdd.count() rdd.count() eventually(timeout(10 seconds), interval(50 milliseconds)) { - go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/jobs") + goToUi(sc, "/jobs") findAll(cssSelector("tbody tr a")).foreach { link => link.text.toLowerCase should include ("count") link.text.toLowerCase should not include "unknown" @@ -321,7 +384,7 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before } sparkUI.attachTab(newTab) eventually(timeout(10 seconds), interval(50 milliseconds)) { - go to (sc.ui.get.appUIAddress.stripSuffix("/")) + goToUi(sc, "") find(cssSelector("""ul li a[href*="jobs"]""")) should not be(None) find(cssSelector("""ul li a[href*="stages"]""")) should not be(None) find(cssSelector("""ul li a[href*="storage"]""")) should not be(None) @@ -330,12 +393,12 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before } eventually(timeout(10 seconds), interval(50 milliseconds)) { // check whether new page exists - go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/foo") + goToUi(sc, "/foo") find(cssSelector("b")).get.text should include ("html magic") } sparkUI.detachTab(newTab) eventually(timeout(10 seconds), interval(50 milliseconds)) { - go to (sc.ui.get.appUIAddress.stripSuffix("/")) + goToUi(sc, "") find(cssSelector("""ul li a[href*="jobs"]""")) should not be(None) find(cssSelector("""ul li a[href*="stages"]""")) should not be(None) find(cssSelector("""ul li a[href*="storage"]""")) should not be(None) @@ -344,7 +407,7 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before } eventually(timeout(10 seconds), interval(50 milliseconds)) { // check new page not exist - go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/foo") + goToUi(sc, "/foo") find(cssSelector("b")) should be(None) } } @@ -371,4 +434,163 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before } } } + + test("stage & job retention") { + val conf = new SparkConf() + .setMaster("local") + .setAppName("test") + .set("spark.ui.enabled", "true") + .set("spark.ui.port", "0") + .set("spark.ui.retainedStages", "3") + .set("spark.ui.retainedJobs", "2") + val sc = new SparkContext(conf) + assert(sc.ui.isDefined) + + withSpark(sc) { sc => + // run a few jobs & stages ... + (0 until 5).foreach { idx => + // NOTE: if we reverse the order, things don't really behave nicely + // we lose the stage for a job we keep, and then the job doesn't know + // about its last stage + sc.parallelize(idx to (idx + 3)).map(identity).groupBy(identity).map(identity) + .groupBy(identity).count() + sc.parallelize(idx to (idx + 3)).collect() + } + + val expJobInfo = Seq( + ("9", "collect"), + ("8", "count") + ) + + eventually(timeout(1 second), interval(50 milliseconds)) { + goToUi(sc, "/jobs") + // The completed jobs table should have two rows. The first row will be the most recent job: + find("completed-summary").get.text should be ("Completed Jobs: 10, only showing 2") + find("completed").get.text should be ("Completed Jobs (10, only showing 2)") + val rows = findAll(cssSelector("tbody tr")).toIndexedSeq.map{_.underlying} + rows.size should be (expJobInfo.size) + for { + (row, idx) <- rows.zipWithIndex + columns = row.findElements(By.tagName("td")) + id = columns(0).getText() + description = columns(1).getText() + } { + id should be (expJobInfo(idx)._1) + description should include (expJobInfo(idx)._2) + } + } + + val jobsJson = getJson(sc.ui.get, "jobs") + jobsJson.children.size should be (expJobInfo.size) + for { + (job @ JObject(_),idx) <- jobsJson.children.zipWithIndex + id = (job \ "jobId").extract[String] + name = (job \ "name").extract[String] + } { + withClue(s"idx = $idx; id = $id; name = ${name.substring(0,20)}") { + id should be (expJobInfo(idx)._1) + name should include (expJobInfo(idx)._2) + } + } + + // what about when we query for a job that did exist, but has been cleared? + goToUi(sc, "/jobs/job/?id=7") + find("no-info").get.text should be ("No information to display for job 7") + + val badJob = HistoryServerSuite.getContentAndCode(jsonUrl(sc.ui.get, "jobs/7")) + badJob._1 should be (HttpServletResponse.SC_NOT_FOUND) + badJob._2 should be (None) + badJob._3 should be (Some("unknown job: 7")) + + val expStageInfo = Seq( + ("19", "collect"), + ("18", "count"), + ("17", "groupBy") + ) + + eventually(timeout(1 second), interval(50 milliseconds)) { + goToUi(sc, "/stages") + find("completed-summary").get.text should be ("Completed Stages: 20, only showing 3") + find("completed").get.text should be ("Completed Stages (20, only showing 3)") + val rows = findAll(cssSelector("tbody tr")).toIndexedSeq.map{_.underlying} + rows.size should be (3) + for { + (row, idx) <- rows.zipWithIndex + columns = row.findElements(By.tagName("td")) + id = columns(0).getText() + description = columns(1).getText() + } { + id should be (expStageInfo(idx)._1) + description should include (expStageInfo(idx)._2) + } + } + + val stagesJson = getJson(sc.ui.get, "stages") + stagesJson.children.size should be (3) + for { + (stage @ JObject(_), idx) <- stagesJson.children.zipWithIndex + id = (stage \ "stageId").extract[String] + name = (stage \ "name").extract[String] + } { + id should be (expStageInfo(idx)._1) + name should include (expStageInfo(idx)._2) + } + + // nonexistent stage + + goToUi(sc, "/stages/stage/?id=12&attempt=0") + find("no-info").get.text should be ("No information to display for Stage 12 (Attempt 0)") + val badStage = HistoryServerSuite.getContentAndCode(jsonUrl(sc.ui.get,"stages/12/0")) + badStage._1 should be (HttpServletResponse.SC_NOT_FOUND) + badStage._2 should be (None) + badStage._3 should be (Some("unknown stage: 12")) + + val badAttempt = HistoryServerSuite.getContentAndCode(jsonUrl(sc.ui.get,"stages/19/15")) + badAttempt._1 should be (HttpServletResponse.SC_NOT_FOUND) + badAttempt._2 should be (None) + badAttempt._3 should be (Some("unknown attempt for stage 19. Found attempts: [0]")) + + val badStageAttemptList = HistoryServerSuite.getContentAndCode( + jsonUrl(sc.ui.get, "stages/12")) + badStageAttemptList._1 should be (HttpServletResponse.SC_NOT_FOUND) + badStageAttemptList._2 should be (None) + badStageAttemptList._3 should be (Some("unknown stage: 12")) + } + } + + test("live UI json application list") { + withSpark(newSparkContext()) { sc => + val appListRawJson = HistoryServerSuite.getUrl(new URL( + sc.ui.get.appUIAddress + "/json/v1/applications")) + val appListJsonAst = JsonMethods.parse(appListRawJson) + appListJsonAst.children.length should be (1) + val attempts = (appListJsonAst \ "attempts").children + attempts.size should be (1) + (attempts(0) \ "completed").extract[Boolean] should be (false) + parseDate(attempts(0) \ "startTime") should be (sc.startTime) + parseDate(attempts(0) \ "endTime") should be (-1) + val oneAppJsonAst = getJson(sc.ui.get, "") + oneAppJsonAst should be (appListJsonAst.children(0)) + } + } + + def goToUi(sc: SparkContext, path: String): Unit = { + goToUi(sc.ui.get, path) + } + + def goToUi(ui: SparkUI, path: String): Unit = { + go to (ui.appUIAddress.stripSuffix("/") + path) + } + + def parseDate(json: JValue): Long = { + JacksonMessageWriter.makeISODateFormat.parse(json.extract[String]).getTime + } + + def getJson(ui: SparkUI, path: String): JValue = { + JsonMethods.parse(HistoryServerSuite.getUrl(jsonUrl(ui, path))) + } + + def jsonUrl(ui: SparkUI, path: String): URL = { + new URL(ui.appUIAddress + "/json/v1/applications/test/" + path) + } } diff --git a/docs/monitoring.md b/docs/monitoring.md index 8a85928d6d44d..1e0fc150862fb 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -174,6 +174,80 @@ making it easy to identify slow tasks, data skew, etc. Note that the history server only displays completed Spark jobs. One way to signal the completion of a Spark job is to stop the Spark Context explicitly (`sc.stop()`), or in Python using the `with SparkContext() as sc:` to handle the Spark Context setup and tear down, and still show the job history on the UI. +## REST API + +In addition to viewing the metrics in the UI, they are also available as JSON. This gives developers +an easy way to create new visualizations and monitoring tools for Spark. The JSON is available for +both running applications, and in the history server. The endpoints are mounted at `/json/v1`. Eg., +for the history server, they would typically be accessible at `http://:18080/json/v1`, and +for a running application, at `http://localhost:4040/json/v1`. + +
      {status.blockManagerId.host + ":" + status.blockManagerId.port}{worker.address} - {Utils.bytesToString(status.memUsedByRdd(rddId))} - ({Utils.bytesToString(status.memRemaining)} Remaining) + {Utils.bytesToString(worker.memoryUsed)} + ({Utils.bytesToString(worker.memoryRemaining)} Remaining) {Utils.bytesToString(status.diskUsedByRdd(rddId))}{Utils.bytesToString(worker.diskUsed)}
      {id}{row.blockName} - {block.storageLevel.description} + {row.storageLevel} - {Utils.bytesToString(block.memSize)} + + {Utils.bytesToString(row.memoryUsed)} - {Utils.bytesToString(block.diskSize)} + + {Utils.bytesToString(row.diskUsed)} - {locations.map(l => {l}
      )} + {row.executors.map(l => {l}
      )}
      + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
      EndpointMeaning
      /applicationsA list of all applications
      /applications/[app-id]/jobsA list of all jobs for a given application
      /applications/[app-id]/jobs/[job-id]Details for the given job
      /applications/[app-id]/stagesA list of all stages for a given application
      /applications/[app-id]/stages/[stage-id]A list of all attempts for the given stage
      /applications/[app-id]/stages/[stage-id]/[stage-attempt-id]Details for the given stage attempt
      /applications/[app-id]/stages/[stage-id]/[stage-attempt-id]/taskSummarySummary metrics of all tasks in the given stage attempt
      /applications/[app-id]/stages/[stage-id]/[stage-attempt-id]/taskListA list of all tasks for the given stage attempt
      /applications/[app-id]/executorsA list of all executors for the given application
      /applications/[app-id]/storage/rddA list of stored RDDs for the given application
      /applications/[app-id]/storage/rdd/[rdd-id]Details for the storage status of a given RDD
      + +When running on Yarn, each application has multiple attempts, so `[app-id]` is actually +`[app-id]/[attempt-id]` in all cases. + +These endpoints have been strongly versioned to make it easier to develop applications on top. + In particular, Spark guarantees: + +* Endpoints will never be removed from one version +* Individual fields will never be removed for any given endpoint +* New endpoints may be added +* New fields may be added to existing endpoints +* New versions of the api may be added in the future at a separate endpoint (eg., `json/v2`). New versions are *not* required to be backwards compatible. +* Api versions may be dropped, but only after at least one minor release of co-existing with a new api version + +Note that even when examining the UI of a running applications, the `applications/[app-id]` portion is +still required, though there is only one application available. Eg. to see the list of jobs for the +running app, you would go to `http://localhost:4040/json/v1/applications/[app-id]/jobs`. This is to +keep the paths consistent in both modes. + # Metrics Spark has a configurable metrics system based on the diff --git a/pom.xml b/pom.xml index 12ad7c5be4e40..94cf28332cf47 100644 --- a/pom.xml +++ b/pom.xml @@ -623,6 +623,18 @@ + + com.sun.jersey + jersey-server + 1.9 + ${hadoop.deps.scope} + + + com.sun.jersey + jersey-core + 1.9 + ${hadoop.deps.scope} + org.scala-lang scala-compiler From f5ff4a84c4c75143086aae7d38730156bee35933 Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Fri, 8 May 2015 11:14:39 -0700 Subject: [PATCH 60/75] [SPARK-7383] [ML] Feature Parity in PySpark for ml.features Implemented python wrappers for Scala functions that don't exist in `ml.features` Author: Burak Yavuz Closes #5991 from brkyvz/ml-feat-PR and squashes the following commits: adcca55 [Burak Yavuz] add regex tokenizer to __all__ b91cb44 [Burak Yavuz] addressed comments bd39fd2 [Burak Yavuz] remove addition b82bd7c [Burak Yavuz] Parity in PySpark for ml.features --- .../ml/feature/PolynomialExpansion.scala | 2 +- .../apache/spark/ml/feature/Tokenizer.scala | 2 +- python/pyspark/ml/feature.py | 797 +++++++++++++++++- .../ml/param/_shared_params_code_gen.py | 4 +- python/pyspark/ml/param/shared.py | 89 ++ 5 files changed, 851 insertions(+), 43 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/PolynomialExpansion.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/PolynomialExpansion.scala index 63e190c8aae53..9e6177ca27e4a 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/PolynomialExpansion.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/PolynomialExpansion.scala @@ -31,7 +31,7 @@ import org.apache.spark.sql.types.DataType * which is available at [[http://en.wikipedia.org/wiki/Polynomial_expansion]], "In mathematics, an * expansion of a product of sums expresses it as a sum of products by using the fact that * multiplication distributes over addition". Take a 2-variable feature vector as an example: - * `(x, y)`, if we want to expand it with degree 2, then we get `(x, y, x * x, x * y, y * y)`. + * `(x, y)`, if we want to expand it with degree 2, then we get `(x, x * x, y, x * y, y * y)`. */ @AlphaComponent class PolynomialExpansion extends UnaryTransformer[Vector, Vector, PolynomialExpansion] { diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Tokenizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Tokenizer.scala index 2863b7621526e..649c217b16590 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Tokenizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Tokenizer.scala @@ -42,7 +42,7 @@ class Tokenizer extends UnaryTransformer[String, Seq[String], Tokenizer] { /** * :: AlphaComponent :: - * A regex based tokenizer that extracts tokens either by repeatedly matching the regex(default) + * A regex based tokenizer that extracts tokens either by repeatedly matching the regex(default) * or using it to split the text (set matching to false). Optional parameters also allow filtering * tokens using a minimal length. * It returns an array of strings that can be empty. diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index 705a368192c24..f35bc1463d51b 100644 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -16,61 +16,72 @@ # from pyspark.rdd import ignore_unicode_prefix -from pyspark.ml.param.shared import HasInputCol, HasInputCols, HasOutputCol, HasNumFeatures +from pyspark.ml.param.shared import * from pyspark.ml.util import keyword_only -from pyspark.ml.wrapper import JavaTransformer +from pyspark.ml.wrapper import JavaEstimator, JavaModel, JavaTransformer from pyspark.mllib.common import inherit_doc -__all__ = ['Tokenizer', 'HashingTF', 'VectorAssembler'] +__all__ = ['Binarizer', 'HashingTF', 'IDF', 'IDFModel', 'Normalizer', 'OneHotEncoder', + 'PolynomialExpansion', 'RegexTokenizer', 'StandardScaler', 'StandardScalerModel', + 'StringIndexer', 'StringIndexerModel', 'Tokenizer', 'VectorAssembler', 'VectorIndexer', + 'Word2Vec', 'Word2VecModel'] @inherit_doc -@ignore_unicode_prefix -class Tokenizer(JavaTransformer, HasInputCol, HasOutputCol): +class Binarizer(JavaTransformer, HasInputCol, HasOutputCol): """ - A tokenizer that converts the input string to lowercase and then - splits it by white spaces. + Binarize a column of continuous features given a threshold. - >>> from pyspark.sql import Row - >>> df = sc.parallelize([Row(text="a b c")]).toDF() - >>> tokenizer = Tokenizer(inputCol="text", outputCol="words") - >>> tokenizer.transform(df).head() - Row(text=u'a b c', words=[u'a', u'b', u'c']) - >>> # Change a parameter. - >>> tokenizer.setParams(outputCol="tokens").transform(df).head() - Row(text=u'a b c', tokens=[u'a', u'b', u'c']) - >>> # Temporarily modify a parameter. - >>> tokenizer.transform(df, {tokenizer.outputCol: "words"}).head() - Row(text=u'a b c', words=[u'a', u'b', u'c']) - >>> tokenizer.transform(df).head() - Row(text=u'a b c', tokens=[u'a', u'b', u'c']) - >>> # Must use keyword arguments to specify params. - >>> tokenizer.setParams("text") - Traceback (most recent call last): - ... - TypeError: Method setParams forces keyword arguments. + >>> df = sqlContext.createDataFrame([(0.5,)], ["values"]) + >>> binarizer = Binarizer(threshold=1.0, inputCol="values", outputCol="features") + >>> binarizer.transform(df).head().features + 0.0 + >>> binarizer.setParams(outputCol="freqs").transform(df).head().freqs + 0.0 + >>> params = {binarizer.threshold: -0.5, binarizer.outputCol: "vector"} + >>> binarizer.transform(df, params).head().vector + 1.0 """ - _java_class = "org.apache.spark.ml.feature.Tokenizer" + _java_class = "org.apache.spark.ml.feature.Binarizer" + # a placeholder to make it appear in the generated doc + threshold = Param(Params._dummy(), "threshold", + "threshold in binary classification prediction, in range [0, 1]") @keyword_only - def __init__(self, inputCol=None, outputCol=None): + def __init__(self, threshold=0.0, inputCol=None, outputCol=None): """ - __init__(self, inputCol=None, outputCol=None) + __init__(self, threshold=0.0, inputCol=None, outputCol=None) """ - super(Tokenizer, self).__init__() + super(Binarizer, self).__init__() + self.threshold = Param(self, "threshold", + "threshold in binary classification prediction, in range [0, 1]") + self._setDefault(threshold=0.0) kwargs = self.__init__._input_kwargs self.setParams(**kwargs) @keyword_only - def setParams(self, inputCol=None, outputCol=None): + def setParams(self, threshold=0.0, inputCol=None, outputCol=None): """ - setParams(self, inputCol="input", outputCol="output") - Sets params for this Tokenizer. + setParams(self, threshold=0.0, inputCol=None, outputCol=None) + Sets params for this Binarizer. """ kwargs = self.setParams._input_kwargs return self._set(**kwargs) + def setThreshold(self, value): + """ + Sets the value of :py:attr:`threshold`. + """ + self.paramMap[self.threshold] = value + return self + + def getThreshold(self): + """ + Gets the value of threshold or its default value. + """ + return self.getOrDefault(self.threshold) + @inherit_doc class HashingTF(JavaTransformer, HasInputCol, HasOutputCol, HasNumFeatures): @@ -78,8 +89,7 @@ class HashingTF(JavaTransformer, HasInputCol, HasOutputCol, HasNumFeatures): Maps a sequence of terms to their term frequencies using the hashing trick. - >>> from pyspark.sql import Row - >>> df = sc.parallelize([Row(words=["a", "b", "c"])]).toDF() + >>> df = sqlContext.createDataFrame([(["a", "b", "c"],)], ["words"]) >>> hashingTF = HashingTF(numFeatures=10, inputCol="words", outputCol="features") >>> hashingTF.transform(df).head().features SparseVector(10, {7: 1.0, 8: 1.0, 9: 1.0}) @@ -112,13 +122,520 @@ def setParams(self, numFeatures=1 << 18, inputCol=None, outputCol=None): return self._set(**kwargs) +@inherit_doc +class IDF(JavaEstimator, HasInputCol, HasOutputCol): + """ + Compute the Inverse Document Frequency (IDF) given a collection of documents. + + >>> from pyspark.mllib.linalg import DenseVector + >>> df = sqlContext.createDataFrame([(DenseVector([1.0, 2.0]),), + ... (DenseVector([0.0, 1.0]),), (DenseVector([3.0, 0.2]),)], ["tf"]) + >>> idf = IDF(minDocFreq=3, inputCol="tf", outputCol="idf") + >>> idf.fit(df).transform(df).head().idf + DenseVector([0.0, 0.0]) + >>> idf.setParams(outputCol="freqs").fit(df).transform(df).collect()[1].freqs + DenseVector([0.0, 0.0]) + >>> params = {idf.minDocFreq: 1, idf.outputCol: "vector"} + >>> idf.fit(df, params).transform(df).head().vector + DenseVector([0.2877, 0.0]) + """ + + _java_class = "org.apache.spark.ml.feature.IDF" + + # a placeholder to make it appear in the generated doc + minDocFreq = Param(Params._dummy(), "minDocFreq", + "minimum of documents in which a term should appear for filtering") + + @keyword_only + def __init__(self, minDocFreq=0, inputCol=None, outputCol=None): + """ + __init__(self, minDocFreq=0, inputCol=None, outputCol=None) + """ + super(IDF, self).__init__() + self.minDocFreq = Param(self, "minDocFreq", + "minimum of documents in which a term should appear for filtering") + self._setDefault(minDocFreq=0) + kwargs = self.__init__._input_kwargs + self.setParams(**kwargs) + + @keyword_only + def setParams(self, minDocFreq=0, inputCol=None, outputCol=None): + """ + setParams(self, minDocFreq=0, inputCol=None, outputCol=None) + Sets params for this IDF. + """ + kwargs = self.setParams._input_kwargs + return self._set(**kwargs) + + def setMinDocFreq(self, value): + """ + Sets the value of :py:attr:`minDocFreq`. + """ + self.paramMap[self.minDocFreq] = value + return self + + def getMinDocFreq(self): + """ + Gets the value of minDocFreq or its default value. + """ + return self.getOrDefault(self.minDocFreq) + + +class IDFModel(JavaModel): + """ + Model fitted by IDF. + """ + + +@inherit_doc +class Normalizer(JavaTransformer, HasInputCol, HasOutputCol): + """ + Normalize a vector to have unit norm using the given p-norm. + + >>> from pyspark.mllib.linalg import Vectors + >>> svec = Vectors.sparse(4, {1: 4.0, 3: 3.0}) + >>> df = sqlContext.createDataFrame([(Vectors.dense([3.0, -4.0]), svec)], ["dense", "sparse"]) + >>> normalizer = Normalizer(p=2.0, inputCol="dense", outputCol="features") + >>> normalizer.transform(df).head().features + DenseVector([0.6, -0.8]) + >>> normalizer.setParams(inputCol="sparse", outputCol="freqs").transform(df).head().freqs + SparseVector(4, {1: 0.8, 3: 0.6}) + >>> params = {normalizer.p: 1.0, normalizer.inputCol: "dense", normalizer.outputCol: "vector"} + >>> normalizer.transform(df, params).head().vector + DenseVector([0.4286, -0.5714]) + """ + + # a placeholder to make it appear in the generated doc + p = Param(Params._dummy(), "p", "the p norm value.") + + _java_class = "org.apache.spark.ml.feature.Normalizer" + + @keyword_only + def __init__(self, p=2.0, inputCol=None, outputCol=None): + """ + __init__(self, p=2.0, inputCol=None, outputCol=None) + """ + super(Normalizer, self).__init__() + self.p = Param(self, "p", "the p norm value.") + self._setDefault(p=2.0) + kwargs = self.__init__._input_kwargs + self.setParams(**kwargs) + + @keyword_only + def setParams(self, p=2.0, inputCol=None, outputCol=None): + """ + setParams(self, p=2.0, inputCol=None, outputCol=None) + Sets params for this Normalizer. + """ + kwargs = self.setParams._input_kwargs + return self._set(**kwargs) + + def setP(self, value): + """ + Sets the value of :py:attr:`p`. + """ + self.paramMap[self.p] = value + return self + + def getP(self): + """ + Gets the value of p or its default value. + """ + return self.getOrDefault(self.p) + + +@inherit_doc +class OneHotEncoder(JavaTransformer, HasInputCol, HasOutputCol): + """ + A one-hot encoder that maps a column of label indices to a column of binary vectors, with + at most a single one-value. By default, the binary vector has an element for each category, so + with 5 categories, an input value of 2.0 would map to an output vector of + (0.0, 0.0, 1.0, 0.0, 0.0). If includeFirst is set to false, the first category is omitted, so + the output vector for the previous example would be (0.0, 1.0, 0.0, 0.0) and an input value + of 0.0 would map to a vector of all zeros. Including the first category makes the vector columns + linearly dependent because they sum up to one. + + TODO: This method requires the use of StringIndexer first. Decouple them. + + >>> stringIndexer = StringIndexer(inputCol="label", outputCol="indexed") + >>> model = stringIndexer.fit(stringIndDf) + >>> td = model.transform(stringIndDf) + >>> encoder = OneHotEncoder(includeFirst=False, inputCol="indexed", outputCol="features") + >>> encoder.transform(td).head().features + SparseVector(2, {}) + >>> encoder.setParams(outputCol="freqs").transform(td).head().freqs + SparseVector(2, {}) + >>> params = {encoder.includeFirst: True, encoder.outputCol: "test"} + >>> encoder.transform(td, params).head().test + SparseVector(3, {0: 1.0}) + """ + + _java_class = "org.apache.spark.ml.feature.OneHotEncoder" + + # a placeholder to make it appear in the generated doc + includeFirst = Param(Params._dummy(), "includeFirst", "include first category") + + @keyword_only + def __init__(self, includeFirst=True, inputCol=None, outputCol=None): + """ + __init__(self, includeFirst=True, inputCol=None, outputCol=None) + """ + super(OneHotEncoder, self).__init__() + self.includeFirst = Param(self, "includeFirst", "include first category") + self._setDefault(includeFirst=True) + kwargs = self.__init__._input_kwargs + self.setParams(**kwargs) + + @keyword_only + def setParams(self, includeFirst=True, inputCol=None, outputCol=None): + """ + setParams(self, includeFirst=True, inputCol=None, outputCol=None) + Sets params for this OneHotEncoder. + """ + kwargs = self.setParams._input_kwargs + return self._set(**kwargs) + + def setIncludeFirst(self, value): + """ + Sets the value of :py:attr:`includeFirst`. + """ + self.paramMap[self.includeFirst] = value + return self + + def getIncludeFirst(self): + """ + Gets the value of includeFirst or its default value. + """ + return self.getOrDefault(self.includeFirst) + + +@inherit_doc +class PolynomialExpansion(JavaTransformer, HasInputCol, HasOutputCol): + """ + Perform feature expansion in a polynomial space. As said in wikipedia of Polynomial Expansion, + which is available at `http://en.wikipedia.org/wiki/Polynomial_expansion`, "In mathematics, an + expansion of a product of sums expresses it as a sum of products by using the fact that + multiplication distributes over addition". Take a 2-variable feature vector as an example: + `(x, y)`, if we want to expand it with degree 2, then we get `(x, x * x, y, x * y, y * y)`. + + >>> from pyspark.mllib.linalg import Vectors + >>> df = sqlContext.createDataFrame([(Vectors.dense([0.5, 2.0]),)], ["dense"]) + >>> px = PolynomialExpansion(degree=2, inputCol="dense", outputCol="expanded") + >>> px.transform(df).head().expanded + DenseVector([0.5, 0.25, 2.0, 1.0, 4.0]) + >>> px.setParams(outputCol="test").transform(df).head().test + DenseVector([0.5, 0.25, 2.0, 1.0, 4.0]) + """ + + _java_class = "org.apache.spark.ml.feature.PolynomialExpansion" + + # a placeholder to make it appear in the generated doc + degree = Param(Params._dummy(), "degree", "the polynomial degree to expand (>= 1)") + + @keyword_only + def __init__(self, degree=2, inputCol=None, outputCol=None): + """ + __init__(self, degree=2, inputCol=None, outputCol=None) + """ + super(PolynomialExpansion, self).__init__() + self.degree = Param(self, "degree", "the polynomial degree to expand (>= 1)") + self._setDefault(degree=2) + kwargs = self.__init__._input_kwargs + self.setParams(**kwargs) + + @keyword_only + def setParams(self, degree=2, inputCol=None, outputCol=None): + """ + setParams(self, degree=2, inputCol=None, outputCol=None) + Sets params for this PolynomialExpansion. + """ + kwargs = self.setParams._input_kwargs + return self._set(**kwargs) + + def setDegree(self, value): + """ + Sets the value of :py:attr:`degree`. + """ + self.paramMap[self.degree] = value + return self + + def getDegree(self): + """ + Gets the value of degree or its default value. + """ + return self.getOrDefault(self.degree) + + +@inherit_doc +@ignore_unicode_prefix +class RegexTokenizer(JavaTransformer, HasInputCol, HasOutputCol): + """ + A regex based tokenizer that extracts tokens either by repeatedly matching the regex(default) + or using it to split the text (set matching to false). Optional parameters also allow filtering + tokens using a minimal length. + It returns an array of strings that can be empty. + + >>> df = sqlContext.createDataFrame([("a b c",)], ["text"]) + >>> reTokenizer = RegexTokenizer(inputCol="text", outputCol="words") + >>> reTokenizer.transform(df).head() + Row(text=u'a b c', words=[u'a', u'b', u'c']) + >>> # Change a parameter. + >>> reTokenizer.setParams(outputCol="tokens").transform(df).head() + Row(text=u'a b c', tokens=[u'a', u'b', u'c']) + >>> # Temporarily modify a parameter. + >>> reTokenizer.transform(df, {reTokenizer.outputCol: "words"}).head() + Row(text=u'a b c', words=[u'a', u'b', u'c']) + >>> reTokenizer.transform(df).head() + Row(text=u'a b c', tokens=[u'a', u'b', u'c']) + >>> # Must use keyword arguments to specify params. + >>> reTokenizer.setParams("text") + Traceback (most recent call last): + ... + TypeError: Method setParams forces keyword arguments. + """ + + _java_class = "org.apache.spark.ml.feature.RegexTokenizer" + # a placeholder to make it appear in the generated doc + minTokenLength = Param(Params._dummy(), "minTokenLength", "minimum token length (>= 0)") + gaps = Param(Params._dummy(), "gaps", "Set regex to match gaps or tokens") + pattern = Param(Params._dummy(), "pattern", "regex pattern used for tokenizing") + + @keyword_only + def __init__(self, minTokenLength=1, gaps=False, pattern="\\p{L}+|[^\\p{L}\\s]+", + inputCol=None, outputCol=None): + """ + __init__(self, minTokenLength=1, gaps=False, pattern="\\p{L}+|[^\\p{L}\\s]+", + inputCol=None, outputCol=None) + """ + super(RegexTokenizer, self).__init__() + self.minTokenLength = Param(self, "minLength", "minimum token length (>= 0)") + self.gaps = Param(self, "gaps", "Set regex to match gaps or tokens") + self.pattern = Param(self, "pattern", "regex pattern used for tokenizing") + self._setDefault(minTokenLength=1, gaps=False, pattern="\\p{L}+|[^\\p{L}\\s]+") + kwargs = self.__init__._input_kwargs + self.setParams(**kwargs) + + @keyword_only + def setParams(self, minTokenLength=1, gaps=False, pattern="\\p{L}+|[^\\p{L}\\s]+", + inputCol=None, outputCol=None): + """ + setParams(self, minTokenLength=1, gaps=False, pattern="\\p{L}+|[^\\p{L}\\s]+", + inputCol="input", outputCol="output") + Sets params for this RegexTokenizer. + """ + kwargs = self.setParams._input_kwargs + return self._set(**kwargs) + + def setMinTokenLength(self, value): + """ + Sets the value of :py:attr:`minTokenLength`. + """ + self.paramMap[self.minTokenLength] = value + return self + + def getMinTokenLength(self): + """ + Gets the value of minTokenLength or its default value. + """ + return self.getOrDefault(self.minTokenLength) + + def setGaps(self, value): + """ + Sets the value of :py:attr:`gaps`. + """ + self.paramMap[self.gaps] = value + return self + + def getGaps(self): + """ + Gets the value of gaps or its default value. + """ + return self.getOrDefault(self.gaps) + + def setPattern(self, value): + """ + Sets the value of :py:attr:`pattern`. + """ + self.paramMap[self.pattern] = value + return self + + def getPattern(self): + """ + Gets the value of pattern or its default value. + """ + return self.getOrDefault(self.pattern) + + +@inherit_doc +class StandardScaler(JavaEstimator, HasInputCol, HasOutputCol): + """ + Standardizes features by removing the mean and scaling to unit variance using column summary + statistics on the samples in the training set. + + >>> from pyspark.mllib.linalg import Vectors + >>> df = sqlContext.createDataFrame([(Vectors.dense([0.0]),), (Vectors.dense([2.0]),)], ["a"]) + >>> standardScaler = StandardScaler(inputCol="a", outputCol="scaled") + >>> model = standardScaler.fit(df) + >>> model.transform(df).collect()[1].scaled + DenseVector([1.4142]) + """ + + _java_class = "org.apache.spark.ml.feature.StandardScaler" + + # a placeholder to make it appear in the generated doc + withMean = Param(Params._dummy(), "withMean", "Center data with mean") + withStd = Param(Params._dummy(), "withStd", "Scale to unit standard deviation") + + @keyword_only + def __init__(self, withMean=False, withStd=True, inputCol=None, outputCol=None): + """ + __init__(self, withMean=False, withStd=True, inputCol=None, outputCol=None) + """ + super(StandardScaler, self).__init__() + self.withMean = Param(self, "withMean", "Center data with mean") + self.withStd = Param(self, "withStd", "Scale to unit standard deviation") + self._setDefault(withMean=False, withStd=True) + kwargs = self.__init__._input_kwargs + self.setParams(**kwargs) + + @keyword_only + def setParams(self, withMean=False, withStd=True, inputCol=None, outputCol=None): + """ + setParams(self, withMean=False, withStd=True, inputCol=None, outputCol=None) + Sets params for this StandardScaler. + """ + kwargs = self.setParams._input_kwargs + return self._set(**kwargs) + + def setWithMean(self, value): + """ + Sets the value of :py:attr:`withMean`. + """ + self.paramMap[self.withMean] = value + return self + + def getWithMean(self): + """ + Gets the value of withMean or its default value. + """ + return self.getOrDefault(self.withMean) + + def setWithStd(self, value): + """ + Sets the value of :py:attr:`withStd`. + """ + self.paramMap[self.withStd] = value + return self + + def getWithStd(self): + """ + Gets the value of withStd or its default value. + """ + return self.getOrDefault(self.withStd) + + +class StandardScalerModel(JavaModel): + """ + Model fitted by StandardScaler. + """ + + +@inherit_doc +class StringIndexer(JavaEstimator, HasInputCol, HasOutputCol): + """ + A label indexer that maps a string column of labels to an ML column of label indices. + If the input column is numeric, we cast it to string and index the string values. + The indices are in [0, numLabels), ordered by label frequencies. + So the most frequent label gets index 0. + + >>> stringIndexer = StringIndexer(inputCol="label", outputCol="indexed") + >>> model = stringIndexer.fit(stringIndDf) + >>> td = model.transform(stringIndDf) + >>> sorted(set([(i[0], i[1]) for i in td.select(td.id, td.indexed).collect()]), + ... key=lambda x: x[0]) + [(0, 0.0), (1, 2.0), (2, 1.0), (3, 0.0), (4, 0.0), (5, 1.0)] + """ + + _java_class = "org.apache.spark.ml.feature.StringIndexer" + + @keyword_only + def __init__(self, inputCol=None, outputCol=None): + """ + __init__(self, inputCol=None, outputCol=None) + """ + super(StringIndexer, self).__init__() + kwargs = self.__init__._input_kwargs + self.setParams(**kwargs) + + @keyword_only + def setParams(self, inputCol=None, outputCol=None): + """ + setParams(self, inputCol=None, outputCol=None) + Sets params for this StringIndexer. + """ + kwargs = self.setParams._input_kwargs + return self._set(**kwargs) + + +class StringIndexerModel(JavaModel): + """ + Model fitted by StringIndexer. + """ + + +@inherit_doc +@ignore_unicode_prefix +class Tokenizer(JavaTransformer, HasInputCol, HasOutputCol): + """ + A tokenizer that converts the input string to lowercase and then + splits it by white spaces. + + >>> df = sqlContext.createDataFrame([("a b c",)], ["text"]) + >>> tokenizer = Tokenizer(inputCol="text", outputCol="words") + >>> tokenizer.transform(df).head() + Row(text=u'a b c', words=[u'a', u'b', u'c']) + >>> # Change a parameter. + >>> tokenizer.setParams(outputCol="tokens").transform(df).head() + Row(text=u'a b c', tokens=[u'a', u'b', u'c']) + >>> # Temporarily modify a parameter. + >>> tokenizer.transform(df, {tokenizer.outputCol: "words"}).head() + Row(text=u'a b c', words=[u'a', u'b', u'c']) + >>> tokenizer.transform(df).head() + Row(text=u'a b c', tokens=[u'a', u'b', u'c']) + >>> # Must use keyword arguments to specify params. + >>> tokenizer.setParams("text") + Traceback (most recent call last): + ... + TypeError: Method setParams forces keyword arguments. + """ + + _java_class = "org.apache.spark.ml.feature.Tokenizer" + + @keyword_only + def __init__(self, inputCol=None, outputCol=None): + """ + __init__(self, inputCol=None, outputCol=None) + """ + super(Tokenizer, self).__init__() + kwargs = self.__init__._input_kwargs + self.setParams(**kwargs) + + @keyword_only + def setParams(self, inputCol=None, outputCol=None): + """ + setParams(self, inputCol="input", outputCol="output") + Sets params for this Tokenizer. + """ + kwargs = self.setParams._input_kwargs + return self._set(**kwargs) + + @inherit_doc class VectorAssembler(JavaTransformer, HasInputCols, HasOutputCol): """ A feature transformer that merges multiple columns into a vector column. - >>> from pyspark.sql import Row - >>> df = sc.parallelize([Row(a=1, b=0, c=3)]).toDF() + >>> df = sqlContext.createDataFrame([(1, 0, 3)], ["a", "b", "c"]) >>> vecAssembler = VectorAssembler(inputCols=["a", "b", "c"], outputCol="features") >>> vecAssembler.transform(df).head().features DenseVector([1.0, 0.0, 3.0]) @@ -137,7 +654,6 @@ def __init__(self, inputCols=None, outputCol=None): __init__(self, inputCols=None, outputCol=None) """ super(VectorAssembler, self).__init__() - self._setDefault() kwargs = self.__init__._input_kwargs self.setParams(**kwargs) @@ -151,10 +667,208 @@ def setParams(self, inputCols=None, outputCol=None): return self._set(**kwargs) +@inherit_doc +class VectorIndexer(JavaEstimator, HasInputCol, HasOutputCol): + """ + Class for indexing categorical feature columns in a dataset of [[Vector]]. + + This has 2 usage modes: + - Automatically identify categorical features (default behavior) + - This helps process a dataset of unknown vectors into a dataset with some continuous + features and some categorical features. The choice between continuous and categorical + is based upon a maxCategories parameter. + - Set maxCategories to the maximum number of categorical any categorical feature should + have. + - E.g.: Feature 0 has unique values {-1.0, 0.0}, and feature 1 values {1.0, 3.0, 5.0}. + If maxCategories = 2, then feature 0 will be declared categorical and use indices {0, 1}, + and feature 1 will be declared continuous. + - Index all features, if all features are categorical + - If maxCategories is set to be very large, then this will build an index of unique + values for all features. + - Warning: This can cause problems if features are continuous since this will collect ALL + unique values to the driver. + - E.g.: Feature 0 has unique values {-1.0, 0.0}, and feature 1 values {1.0, 3.0, 5.0}. + If maxCategories >= 3, then both features will be declared categorical. + + This returns a model which can transform categorical features to use 0-based indices. + + Index stability: + - This is not guaranteed to choose the same category index across multiple runs. + - If a categorical feature includes value 0, then this is guaranteed to map value 0 to + index 0. This maintains vector sparsity. + - More stability may be added in the future. + + TODO: Future extensions: The following functionality is planned for the future: + - Preserve metadata in transform; if a feature's metadata is already present, + do not recompute. + - Specify certain features to not index, either via a parameter or via existing metadata. + - Add warning if a categorical feature has only 1 category. + - Add option for allowing unknown categories. + + >>> from pyspark.mllib.linalg import Vectors + >>> df = sqlContext.createDataFrame([(Vectors.dense([-1.0, 0.0]),), + ... (Vectors.dense([0.0, 1.0]),), (Vectors.dense([0.0, 2.0]),)], ["a"]) + >>> indexer = VectorIndexer(maxCategories=2, inputCol="a", outputCol="indexed") + >>> model = indexer.fit(df) + >>> model.transform(df).head().indexed + DenseVector([1.0, 0.0]) + >>> indexer.setParams(outputCol="test").fit(df).transform(df).collect()[1].test + DenseVector([0.0, 1.0]) + >>> params = {indexer.maxCategories: 3, indexer.outputCol: "vector"} + >>> model2 = indexer.fit(df, params) + >>> model2.transform(df).head().vector + DenseVector([1.0, 0.0]) + """ + + _java_class = "org.apache.spark.ml.feature.VectorIndexer" + # a placeholder to make it appear in the generated doc + maxCategories = Param(Params._dummy(), "maxCategories", + "Threshold for the number of values a categorical feature can take " + + "(>= 2). If a feature is found to have > maxCategories values, then " + + "it is declared continuous.") + + @keyword_only + def __init__(self, maxCategories=20, inputCol=None, outputCol=None): + """ + __init__(self, maxCategories=20, inputCol=None, outputCol=None) + """ + super(VectorIndexer, self).__init__() + self.maxCategories = Param(self, "maxCategories", + "Threshold for the number of values a categorical feature " + + "can take (>= 2). If a feature is found to have " + + "> maxCategories values, then it is declared continuous.") + self._setDefault(maxCategories=20) + kwargs = self.__init__._input_kwargs + self.setParams(**kwargs) + + @keyword_only + def setParams(self, maxCategories=20, inputCol=None, outputCol=None): + """ + setParams(self, maxCategories=20, inputCol=None, outputCol=None) + Sets params for this VectorIndexer. + """ + kwargs = self.setParams._input_kwargs + return self._set(**kwargs) + + def setMaxCategories(self, value): + """ + Sets the value of :py:attr:`maxCategories`. + """ + self.paramMap[self.maxCategories] = value + return self + + def getMaxCategories(self): + """ + Gets the value of maxCategories or its default value. + """ + return self.getOrDefault(self.maxCategories) + + +@inherit_doc +@ignore_unicode_prefix +class Word2Vec(JavaEstimator, HasStepSize, HasMaxIter, HasSeed, HasInputCol, HasOutputCol): + """ + Word2Vec trains a model of `Map(String, Vector)`, i.e. transforms a word into a code for further + natural language processing or machine learning process. + + >>> sent = ("a b " * 100 + "a c " * 10).split(" ") + >>> doc = sqlContext.createDataFrame([(sent,), (sent,)], ["sentence"]) + >>> model = Word2Vec(vectorSize=5, seed=42, inputCol="sentence", outputCol="model").fit(doc) + >>> model.transform(doc).head().model + DenseVector([-0.0422, -0.5138, -0.2546, 0.6885, 0.276]) + """ + + _java_class = "org.apache.spark.ml.feature.Word2Vec" + # a placeholder to make it appear in the generated doc + vectorSize = Param(Params._dummy(), "vectorSize", + "the dimension of codes after transforming from words") + numPartitions = Param(Params._dummy(), "numPartitions", + "number of partitions for sentences of words") + minCount = Param(Params._dummy(), "minCount", + "the minimum number of times a token must appear to be included in the " + + "word2vec model's vocabulary") + + @keyword_only + def __init__(self, vectorSize=100, minCount=5, numPartitions=1, stepSize=0.025, maxIter=1, + seed=42, inputCol=None, outputCol=None): + """ + __init__(self, vectorSize=100, minCount=5, numPartitions=1, stepSize=0.025, maxIter=1, + seed=42, inputCol=None, outputCol=None) + """ + super(Word2Vec, self).__init__() + self.vectorSize = Param(self, "vectorSize", + "the dimension of codes after transforming from words") + self.numPartitions = Param(self, "numPartitions", + "number of partitions for sentences of words") + self.minCount = Param(self, "minCount", + "the minimum number of times a token must appear to be included " + + "in the word2vec model's vocabulary") + self._setDefault(vectorSize=100, minCount=5, numPartitions=1, stepSize=0.025, maxIter=1, + seed=42) + kwargs = self.__init__._input_kwargs + self.setParams(**kwargs) + + @keyword_only + def setParams(self, vectorSize=100, minCount=5, numPartitions=1, stepSize=0.025, maxIter=1, + seed=42, inputCol=None, outputCol=None): + """ + setParams(self, minCount=5, numPartitions=1, stepSize=0.025, maxIter=1, seed=42, + inputCol=None, outputCol=None) + Sets params for this Word2Vec. + """ + kwargs = self.setParams._input_kwargs + return self._set(**kwargs) + + def setVectorSize(self, value): + """ + Sets the value of :py:attr:`vectorSize`. + """ + self.paramMap[self.vectorSize] = value + return self + + def getVectorSize(self): + """ + Gets the value of vectorSize or its default value. + """ + return self.getOrDefault(self.vectorSize) + + def setNumPartitions(self, value): + """ + Sets the value of :py:attr:`numPartitions`. + """ + self.paramMap[self.numPartitions] = value + return self + + def getNumPartitions(self): + """ + Gets the value of numPartitions or its default value. + """ + return self.getOrDefault(self.numPartitions) + + def setMinCount(self, value): + """ + Sets the value of :py:attr:`minCount`. + """ + self.paramMap[self.minCount] = value + return self + + def getMinCount(self): + """ + Gets the value of minCount or its default value. + """ + return self.getOrDefault(self.minCount) + + +class Word2VecModel(JavaModel): + """ + Model fitted by Word2Vec. + """ + + if __name__ == "__main__": import doctest from pyspark.context import SparkContext - from pyspark.sql import SQLContext + from pyspark.sql import Row, SQLContext globs = globals().copy() # The small batch size here ensures that we see multiple batches, # even in these small test examples: @@ -162,8 +876,11 @@ def setParams(self, inputCols=None, outputCol=None): sqlContext = SQLContext(sc) globs['sc'] = sc globs['sqlContext'] = sqlContext - (failure_count, test_count) = doctest.testmod( - globs=globs, optionflags=doctest.ELLIPSIS) + testData = sc.parallelize([Row(id=0, label="a"), Row(id=1, label="b"), + Row(id=2, label="c"), Row(id=3, label="a"), + Row(id=4, label="a"), Row(id=5, label="c")], 2) + globs['stringIndDf'] = sqlContext.createDataFrame(testData) + (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) sc.stop() if failure_count: exit(-1) diff --git a/python/pyspark/ml/param/_shared_params_code_gen.py b/python/pyspark/ml/param/_shared_params_code_gen.py index c1c8e921dda87..ee901f25847bc 100644 --- a/python/pyspark/ml/param/_shared_params_code_gen.py +++ b/python/pyspark/ml/param/_shared_params_code_gen.py @@ -97,7 +97,9 @@ def get$Name(self): ("inputCol", "input column name", None), ("inputCols", "input column names", None), ("outputCol", "output column name", None), - ("numFeatures", "number of features", None)] + ("seed", "random seed", None), + ("tol", "the convergence tolerance for iterative algorithms", None), + ("stepSize", "Step size to be used for each iteration of optimization.", None)] code = [] for name, doc, defaultValueStr in shared: code.append(_gen_param_code(name, doc, defaultValueStr)) diff --git a/python/pyspark/ml/param/shared.py b/python/pyspark/ml/param/shared.py index aaf80f00085bf..5e7529c1dcf1e 100644 --- a/python/pyspark/ml/param/shared.py +++ b/python/pyspark/ml/param/shared.py @@ -308,3 +308,92 @@ def getNumFeatures(self): Gets the value of numFeatures or its default value. """ return self.getOrDefault(self.numFeatures) + + +class HasSeed(Params): + """ + Mixin for param seed: random seed. + """ + + # a placeholder to make it appear in the generated doc + seed = Param(Params._dummy(), "seed", "random seed") + + def __init__(self): + super(HasSeed, self).__init__() + #: param for random seed + self.seed = Param(self, "seed", "random seed") + if None is not None: + self._setDefault(seed=None) + + def setSeed(self, value): + """ + Sets the value of :py:attr:`seed`. + """ + self.paramMap[self.seed] = value + return self + + def getSeed(self): + """ + Gets the value of seed or its default value. + """ + return self.getOrDefault(self.seed) + + +class HasTol(Params): + """ + Mixin for param tol: the convergence tolerance for iterative algorithms. + """ + + # a placeholder to make it appear in the generated doc + tol = Param(Params._dummy(), "tol", "the convergence tolerance for iterative algorithms") + + def __init__(self): + super(HasTol, self).__init__() + #: param for the convergence tolerance for iterative algorithms + self.tol = Param(self, "tol", "the convergence tolerance for iterative algorithms") + if None is not None: + self._setDefault(tol=None) + + def setTol(self, value): + """ + Sets the value of :py:attr:`tol`. + """ + self.paramMap[self.tol] = value + return self + + def getTol(self): + """ + Gets the value of tol or its default value. + """ + return self.getOrDefault(self.tol) + + +class HasStepSize(Params): + """ + Mixin for param stepSize: Step size to be used for each iteration of optimization.. + """ + + # a placeholder to make it appear in the generated doc + stepSize = Param(Params._dummy(), "stepSize", + "Step size to be used for each iteration of optimization.") + + def __init__(self): + super(HasStepSize, self).__init__() + #: param for Step size to be used for each iteration of optimization. + self.stepSize = Param(self, "stepSize", + "Step size to be used for each iteration of optimization.") + if None is not None: + self._setDefault(stepSize=None) + + def setStepSize(self, value): + """ + Sets the value of :py:attr:`stepSize`. + """ + self.paramMap[self.stepSize] = value + return self + + def getStepSize(self): + """ + Gets the value of stepSize or its default value. + """ + return self.getOrDefault(self.stepSize) From 65afd3ce8b8a0b00f4ea8294eac14b72e964872d Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Fri, 8 May 2015 11:16:04 -0700 Subject: [PATCH 61/75] [SPARK-7474] [MLLIB] update ParamGridBuilder doctest Multiline commands are properly handled in this PR. oefirouz ![screen shot 2015-05-07 at 10 53 25 pm](https://cloud.githubusercontent.com/assets/829644/7531290/02ad2fd4-f50c-11e4-8c04-e58d1a61ad69.png) Author: Xiangrui Meng Closes #6001 from mengxr/SPARK-7474 and squashes the following commits: b94b11d [Xiangrui Meng] update ParamGridBuilder doctest --- python/pyspark/ml/tuning.py | 28 +++++++++++++--------------- 1 file changed, 13 insertions(+), 15 deletions(-) diff --git a/python/pyspark/ml/tuning.py b/python/pyspark/ml/tuning.py index 1e04c37fca53e..28e3727f2c064 100644 --- a/python/pyspark/ml/tuning.py +++ b/python/pyspark/ml/tuning.py @@ -27,24 +27,22 @@ class ParamGridBuilder(object): - """ + r""" Builder for a param grid used in grid search-based model selection. - >>> from classification import LogisticRegression + >>> from pyspark.ml.classification import LogisticRegression >>> lr = LogisticRegression() - >>> output = ParamGridBuilder().baseOn({lr.labelCol: 'l'}) \ - .baseOn([lr.predictionCol, 'p']) \ - .addGrid(lr.regParam, [1.0, 2.0, 3.0]) \ - .addGrid(lr.maxIter, [1, 5]) \ - .addGrid(lr.featuresCol, ['f']) \ - .build() - >>> expected = [ \ -{lr.regParam: 1.0, lr.featuresCol: 'f', lr.maxIter: 1, lr.labelCol: 'l', lr.predictionCol: 'p'}, \ -{lr.regParam: 2.0, lr.featuresCol: 'f', lr.maxIter: 1, lr.labelCol: 'l', lr.predictionCol: 'p'}, \ -{lr.regParam: 3.0, lr.featuresCol: 'f', lr.maxIter: 1, lr.labelCol: 'l', lr.predictionCol: 'p'}, \ -{lr.regParam: 1.0, lr.featuresCol: 'f', lr.maxIter: 5, lr.labelCol: 'l', lr.predictionCol: 'p'}, \ -{lr.regParam: 2.0, lr.featuresCol: 'f', lr.maxIter: 5, lr.labelCol: 'l', lr.predictionCol: 'p'}, \ -{lr.regParam: 3.0, lr.featuresCol: 'f', lr.maxIter: 5, lr.labelCol: 'l', lr.predictionCol: 'p'}] + >>> output = ParamGridBuilder() \ + ... .baseOn({lr.labelCol: 'l'}) \ + ... .baseOn([lr.predictionCol, 'p']) \ + ... .addGrid(lr.regParam, [1.0, 2.0]) \ + ... .addGrid(lr.maxIter, [1, 5]) \ + ... .build() + >>> expected = [ + ... {lr.regParam: 1.0, lr.maxIter: 1, lr.labelCol: 'l', lr.predictionCol: 'p'}, + ... {lr.regParam: 2.0, lr.maxIter: 1, lr.labelCol: 'l', lr.predictionCol: 'p'}, + ... {lr.regParam: 1.0, lr.maxIter: 5, lr.labelCol: 'l', lr.predictionCol: 'p'}, + ... {lr.regParam: 2.0, lr.maxIter: 5, lr.labelCol: 'l', lr.predictionCol: 'p'}] >>> len(output) == len(expected) True >>> all([m in expected for m in output]) From 008a60dd371e76819d8e08ab638cac7b3a48c9fc Mon Sep 17 00:00:00 2001 From: hqzizania Date: Fri, 8 May 2015 11:25:04 -0700 Subject: [PATCH 62/75] [SPARK-6824] Fill the docs for DataFrame API in SparkR This patch also removes the RDD docs from being built as a part of roxygen just by the method to delete " ' '" of " \#' ". Author: hqzizania Author: qhuang Closes #5969 from hqzizania/R1 and squashes the following commits: 6d27696 [qhuang] fixes in NAMESPACE eb4b095 [qhuang] remove more docs 6394579 [qhuang] remove RDD docs in generics.R 6813860 [hqzizania] Fill the docs for DataFrame API in SparkR 857220f [hqzizania] remove the pairRDD docs from being built as a part of roxygen c045d64 [hqzizania] remove the RDD docs from being built as a part of roxygen --- R/pkg/DESCRIPTION | 2 +- R/pkg/NAMESPACE | 4 - R/pkg/R/DataFrame.R | 95 +-- R/pkg/R/RDD.R | 1546 +++++++++++++++++++++--------------------- R/pkg/R/SQLContext.R | 64 +- R/pkg/R/broadcast.R | 64 +- R/pkg/R/context.R | 240 +++---- R/pkg/R/generics.R | 318 ++++----- R/pkg/R/pairRDD.R | 886 ++++++++++++------------ 9 files changed, 1610 insertions(+), 1609 deletions(-) diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index 1c1779a763c7e..efc85bbc4b316 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -15,11 +15,11 @@ Suggests: Description: R frontend for Spark License: Apache License (== 2.0) Collate: + 'schema.R' 'generics.R' 'jobj.R' 'RDD.R' 'pairRDD.R' - 'schema.R' 'column.R' 'group.R' 'DataFrame.R' diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE index 3fb92be0940b7..7611f479a628b 100644 --- a/R/pkg/NAMESPACE +++ b/R/pkg/NAMESPACE @@ -26,7 +26,6 @@ exportMethods("cache", "intersect", "isLocal", "join", - "length", "limit", "orderBy", "names", @@ -101,9 +100,6 @@ export("cacheTable", "tables", "uncacheTable") -export("sparkRSQL.init", - "sparkRHive.init") - export("structField", "structField.jobj", "structField.character", diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index 47d92f141cc7d..354642e7bc307 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -45,6 +45,9 @@ setMethod("initialize", "DataFrame", function(.Object, sdf, isCached) { #' @rdname DataFrame #' @export +#' +#' @param sdf A Java object reference to the backing Scala DataFrame +#' @param isCached TRUE if the dataFrame is cached dataFrame <- function(sdf, isCached = FALSE) { new("DataFrame", sdf, isCached) } @@ -244,7 +247,7 @@ setMethod("columns", }) #' @rdname columns -#' @export +#' @aliases names,DataFrame,function-method setMethod("names", signature(x = "DataFrame"), function(x) { @@ -399,23 +402,23 @@ setMethod("repartition", dataFrame(sdf) }) -#' toJSON -#' -#' Convert the rows of a DataFrame into JSON objects and return an RDD where -#' each element contains a JSON string. -#' -#' @param x A SparkSQL DataFrame -#' @return A StringRRDD of JSON objects -#' @rdname tojson -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' sqlCtx <- sparkRSQL.init(sc) -#' path <- "path/to/file.json" -#' df <- jsonFile(sqlCtx, path) -#' newRDD <- toJSON(df) -#'} +# toJSON +# +# Convert the rows of a DataFrame into JSON objects and return an RDD where +# each element contains a JSON string. +# +#@param x A SparkSQL DataFrame +# @return A StringRRDD of JSON objects +# @rdname tojson +# @export +# @examples +#\dontrun{ +# sc <- sparkR.init() +# sqlCtx <- sparkRSQL.init(sc) +# path <- "path/to/file.json" +# df <- jsonFile(sqlCtx, path) +# newRDD <- toJSON(df) +#} setMethod("toJSON", signature(x = "DataFrame"), function(x) { @@ -578,8 +581,8 @@ setMethod("limit", dataFrame(res) }) -# Take the first NUM rows of a DataFrame and return a the results as a data.frame - +#' Take the first NUM rows of a DataFrame and return a the results as a data.frame +#' #' @rdname take #' @export #' @examples @@ -644,22 +647,22 @@ setMethod("first", take(x, 1) }) -#' toRDD() -#' -#' Converts a Spark DataFrame to an RDD while preserving column names. -#' -#' @param x A Spark DataFrame -#' -#' @rdname DataFrame -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' sqlCtx <- sparkRSQL.init(sc) -#' path <- "path/to/file.json" -#' df <- jsonFile(sqlCtx, path) -#' rdd <- toRDD(df) -#' } +# toRDD() +# +# Converts a Spark DataFrame to an RDD while preserving column names. +# +# @param x A Spark DataFrame +# +# @rdname DataFrame +# @export +# @examples +#\dontrun{ +# sc <- sparkR.init() +# sqlCtx <- sparkRSQL.init(sc) +# path <- "path/to/file.json" +# df <- jsonFile(sqlCtx, path) +# rdd <- toRDD(df) +# } setMethod("toRDD", signature(x = "DataFrame"), function(x) { @@ -706,6 +709,7 @@ setMethod("groupBy", #' #' Compute aggregates by specifying a list of columns #' +#' @param x a DataFrame #' @rdname DataFrame #' @export setMethod("agg", @@ -721,7 +725,7 @@ setMethod("agg", # the requested map function. # ################################################################################### -#' @rdname lapply +# @rdname lapply setMethod("lapply", signature(X = "DataFrame", FUN = "function"), function(X, FUN) { @@ -729,14 +733,14 @@ setMethod("lapply", lapply(rdd, FUN) }) -#' @rdname lapply +# @rdname lapply setMethod("map", signature(X = "DataFrame", FUN = "function"), function(X, FUN) { lapply(X, FUN) }) -#' @rdname flatMap +# @rdname flatMap setMethod("flatMap", signature(X = "DataFrame", FUN = "function"), function(X, FUN) { @@ -744,7 +748,7 @@ setMethod("flatMap", flatMap(rdd, FUN) }) -#' @rdname lapplyPartition +# @rdname lapplyPartition setMethod("lapplyPartition", signature(X = "DataFrame", FUN = "function"), function(X, FUN) { @@ -752,14 +756,14 @@ setMethod("lapplyPartition", lapplyPartition(rdd, FUN) }) -#' @rdname lapplyPartition +# @rdname lapplyPartition setMethod("mapPartitions", signature(X = "DataFrame", FUN = "function"), function(X, FUN) { lapplyPartition(X, FUN) }) -#' @rdname foreach +# @rdname foreach setMethod("foreach", signature(x = "DataFrame", func = "function"), function(x, func) { @@ -767,7 +771,7 @@ setMethod("foreach", foreach(rdd, func) }) -#' @rdname foreach +# @rdname foreach setMethod("foreachPartition", signature(x = "DataFrame", func = "function"), function(x, func) { @@ -788,6 +792,7 @@ setMethod("$", signature(x = "DataFrame"), getColumn(x, name) }) +#' @rdname select setMethod("$<-", signature(x = "DataFrame"), function(x, name, value) { stopifnot(class(value) == "Column" || is.null(value)) @@ -1009,7 +1014,7 @@ setMethod("sortDF", }) #' @rdname sortDF -#' @export +#' @aliases orderBy,DataFrame,function-method setMethod("orderBy", signature(x = "DataFrame", col = "characterOrColumn"), function(x, col) { @@ -1046,7 +1051,7 @@ setMethod("filter", }) #' @rdname filter -#' @export +#' @aliases where,DataFrame,function-method setMethod("where", signature(x = "DataFrame", condition = "characterOrColumn"), function(x, condition) { diff --git a/R/pkg/R/RDD.R b/R/pkg/R/RDD.R index d1018c2361670..73999a6737032 100644 --- a/R/pkg/R/RDD.R +++ b/R/pkg/R/RDD.R @@ -19,16 +19,16 @@ setOldClass("jobj") -#' @title S4 class that represents an RDD -#' @description RDD can be created using functions like -#' \code{parallelize}, \code{textFile} etc. -#' @rdname RDD -#' @seealso parallelize, textFile -#' -#' @slot env An R environment that stores bookkeeping states of the RDD -#' @slot jrdd Java object reference to the backing JavaRDD -#' to an RDD -#' @export +# @title S4 class that represents an RDD +# @description RDD can be created using functions like +# \code{parallelize}, \code{textFile} etc. +# @rdname RDD +# @seealso parallelize, textFile +# +# @slot env An R environment that stores bookkeeping states of the RDD +# @slot jrdd Java object reference to the backing JavaRDD +# to an RDD +# @export setClass("RDD", slots = list(env = "environment", jrdd = "jobj")) @@ -108,14 +108,14 @@ setMethod("initialize", "PipelinedRDD", function(.Object, prev, func, jrdd_val) .Object }) -#' @rdname RDD -#' @export -#' -#' @param jrdd Java object reference to the backing JavaRDD -#' @param serializedMode Use "byte" if the RDD stores data serialized in R, "string" if the RDD -#' stores strings, and "row" if the RDD stores the rows of a DataFrame -#' @param isCached TRUE if the RDD is cached -#' @param isCheckpointed TRUE if the RDD has been checkpointed +# @rdname RDD +# @export +# +# @param jrdd Java object reference to the backing JavaRDD +# @param serializedMode Use "byte" if the RDD stores data serialized in R, "string" if the RDD +# stores strings, and "row" if the RDD stores the rows of a DataFrame +# @param isCached TRUE if the RDD is cached +# @param isCheckpointed TRUE if the RDD has been checkpointed RDD <- function(jrdd, serializedMode = "byte", isCached = FALSE, isCheckpointed = FALSE) { new("RDD", jrdd, serializedMode, isCached, isCheckpointed) @@ -200,19 +200,19 @@ setValidity("RDD", ############ Actions and Transformations ############ -#' Persist an RDD -#' -#' Persist this RDD with the default storage level (MEMORY_ONLY). -#' -#' @param x The RDD to cache -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10, 2L) -#' cache(rdd) -#'} -#' @rdname cache-methods -#' @aliases cache,RDD-method +# Persist an RDD +# +# Persist this RDD with the default storage level (MEMORY_ONLY). +# +# @param x The RDD to cache +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, 1:10, 2L) +# cache(rdd) +#} +# @rdname cache-methods +# @aliases cache,RDD-method setMethod("cache", signature(x = "RDD"), function(x) { @@ -221,22 +221,22 @@ setMethod("cache", x }) -#' Persist an RDD -#' -#' Persist this RDD with the specified storage level. For details of the -#' supported storage levels, refer to -#' http://spark.apache.org/docs/latest/programming-guide.html#rdd-persistence. -#' -#' @param x The RDD to persist -#' @param newLevel The new storage level to be assigned -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10, 2L) -#' persist(rdd, "MEMORY_AND_DISK") -#'} -#' @rdname persist -#' @aliases persist,RDD-method +# Persist an RDD +# +# Persist this RDD with the specified storage level. For details of the +# supported storage levels, refer to +# http://spark.apache.org/docs/latest/programming-guide.html#rdd-persistence. +# +# @param x The RDD to persist +# @param newLevel The new storage level to be assigned +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, 1:10, 2L) +# persist(rdd, "MEMORY_AND_DISK") +#} +# @rdname persist +# @aliases persist,RDD-method setMethod("persist", signature(x = "RDD", newLevel = "character"), function(x, newLevel) { @@ -245,21 +245,21 @@ setMethod("persist", x }) -#' Unpersist an RDD -#' -#' Mark the RDD as non-persistent, and remove all blocks for it from memory and -#' disk. -#' -#' @param x The RDD to unpersist -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10, 2L) -#' cache(rdd) # rdd@@env$isCached == TRUE -#' unpersist(rdd) # rdd@@env$isCached == FALSE -#'} -#' @rdname unpersist-methods -#' @aliases unpersist,RDD-method +# Unpersist an RDD +# +# Mark the RDD as non-persistent, and remove all blocks for it from memory and +# disk. +# +# @param x The RDD to unpersist +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, 1:10, 2L) +# cache(rdd) # rdd@@env$isCached == TRUE +# unpersist(rdd) # rdd@@env$isCached == FALSE +#} +# @rdname unpersist-methods +# @aliases unpersist,RDD-method setMethod("unpersist", signature(x = "RDD"), function(x) { @@ -268,24 +268,24 @@ setMethod("unpersist", x }) -#' Checkpoint an RDD -#' -#' Mark this RDD for checkpointing. It will be saved to a file inside the -#' checkpoint directory set with setCheckpointDir() and all references to its -#' parent RDDs will be removed. This function must be called before any job has -#' been executed on this RDD. It is strongly recommended that this RDD is -#' persisted in memory, otherwise saving it on a file will require recomputation. -#' -#' @param x The RDD to checkpoint -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' setCheckpointDir(sc, "checkpoint") -#' rdd <- parallelize(sc, 1:10, 2L) -#' checkpoint(rdd) -#'} -#' @rdname checkpoint-methods -#' @aliases checkpoint,RDD-method +# Checkpoint an RDD +# +# Mark this RDD for checkpointing. It will be saved to a file inside the +# checkpoint directory set with setCheckpointDir() and all references to its +# parent RDDs will be removed. This function must be called before any job has +# been executed on this RDD. It is strongly recommended that this RDD is +# persisted in memory, otherwise saving it on a file will require recomputation. +# +# @param x The RDD to checkpoint +# @examples +#\dontrun{ +# sc <- sparkR.init() +# setCheckpointDir(sc, "checkpoint") +# rdd <- parallelize(sc, 1:10, 2L) +# checkpoint(rdd) +#} +# @rdname checkpoint-methods +# @aliases checkpoint,RDD-method setMethod("checkpoint", signature(x = "RDD"), function(x) { @@ -295,18 +295,18 @@ setMethod("checkpoint", x }) -#' Gets the number of partitions of an RDD -#' -#' @param x A RDD. -#' @return the number of partitions of rdd as an integer. -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10, 2L) -#' numPartitions(rdd) # 2L -#'} -#' @rdname numPartitions -#' @aliases numPartitions,RDD-method +# Gets the number of partitions of an RDD +# +# @param x A RDD. +# @return the number of partitions of rdd as an integer. +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, 1:10, 2L) +# numPartitions(rdd) # 2L +#} +# @rdname numPartitions +# @aliases numPartitions,RDD-method setMethod("numPartitions", signature(x = "RDD"), function(x) { @@ -315,24 +315,24 @@ setMethod("numPartitions", callJMethod(partitions, "size") }) -#' Collect elements of an RDD -#' -#' @description -#' \code{collect} returns a list that contains all of the elements in this RDD. -#' -#' @param x The RDD to collect -#' @param ... Other optional arguments to collect -#' @param flatten FALSE if the list should not flattened -#' @return a list containing elements in the RDD -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10, 2L) -#' collect(rdd) # list from 1 to 10 -#' collectPartition(rdd, 0L) # list from 1 to 5 -#'} -#' @rdname collect-methods -#' @aliases collect,RDD-method +# Collect elements of an RDD +# +# @description +# \code{collect} returns a list that contains all of the elements in this RDD. +# +# @param x The RDD to collect +# @param ... Other optional arguments to collect +# @param flatten FALSE if the list should not flattened +# @return a list containing elements in the RDD +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, 1:10, 2L) +# collect(rdd) # list from 1 to 10 +# collectPartition(rdd, 0L) # list from 1 to 5 +#} +# @rdname collect-methods +# @aliases collect,RDD-method setMethod("collect", signature(x = "RDD"), function(x, flatten = TRUE) { @@ -343,12 +343,12 @@ setMethod("collect", }) -#' @description -#' \code{collectPartition} returns a list that contains all of the elements -#' in the specified partition of the RDD. -#' @param partitionId the partition to collect (starts from 0) -#' @rdname collect-methods -#' @aliases collectPartition,integer,RDD-method +# @description +# \code{collectPartition} returns a list that contains all of the elements +# in the specified partition of the RDD. +# @param partitionId the partition to collect (starts from 0) +# @rdname collect-methods +# @aliases collectPartition,integer,RDD-method setMethod("collectPartition", signature(x = "RDD", partitionId = "integer"), function(x, partitionId) { @@ -361,17 +361,17 @@ setMethod("collectPartition", serializedMode = getSerializedMode(x)) }) -#' @description -#' \code{collectAsMap} returns a named list as a map that contains all of the elements -#' in a key-value pair RDD. -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list(list(1, 2), list(3, 4)), 2L) -#' collectAsMap(rdd) # list(`1` = 2, `3` = 4) -#'} -#' @rdname collect-methods -#' @aliases collectAsMap,RDD-method +# @description +# \code{collectAsMap} returns a named list as a map that contains all of the elements +# in a key-value pair RDD. +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, list(list(1, 2), list(3, 4)), 2L) +# collectAsMap(rdd) # list(`1` = 2, `3` = 4) +#} +# @rdname collect-methods +# @aliases collectAsMap,RDD-method setMethod("collectAsMap", signature(x = "RDD"), function(x) { @@ -381,19 +381,19 @@ setMethod("collectAsMap", as.list(map) }) -#' Return the number of elements in the RDD. -#' -#' @param x The RDD to count -#' @return number of elements in the RDD. -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10) -#' count(rdd) # 10 -#' length(rdd) # Same as count -#'} -#' @rdname count -#' @aliases count,RDD-method +# Return the number of elements in the RDD. +# +# @param x The RDD to count +# @return number of elements in the RDD. +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, 1:10) +# count(rdd) # 10 +# length(rdd) # Same as count +#} +# @rdname count +# @aliases count,RDD-method setMethod("count", signature(x = "RDD"), function(x) { @@ -405,31 +405,31 @@ setMethod("count", sum(as.integer(vals)) }) -#' Return the number of elements in the RDD -#' @export -#' @rdname count +# Return the number of elements in the RDD +# @export +# @rdname count setMethod("length", signature(x = "RDD"), function(x) { count(x) }) -#' Return the count of each unique value in this RDD as a list of -#' (value, count) pairs. -#' -#' Same as countByValue in Spark. -#' -#' @param x The RDD to count -#' @return list of (value, count) pairs, where count is number of each unique -#' value in rdd. -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, c(1,2,3,2,1)) -#' countByValue(rdd) # (1,2L), (2,2L), (3,1L) -#'} -#' @rdname countByValue -#' @aliases countByValue,RDD-method +# Return the count of each unique value in this RDD as a list of +# (value, count) pairs. +# +# Same as countByValue in Spark. +# +# @param x The RDD to count +# @return list of (value, count) pairs, where count is number of each unique +# value in rdd. +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, c(1,2,3,2,1)) +# countByValue(rdd) # (1,2L), (2,2L), (3,1L) +#} +# @rdname countByValue +# @aliases countByValue,RDD-method setMethod("countByValue", signature(x = "RDD"), function(x) { @@ -437,23 +437,23 @@ setMethod("countByValue", collect(reduceByKey(ones, `+`, numPartitions(x))) }) -#' Apply a function to all elements -#' -#' This function creates a new RDD by applying the given transformation to all -#' elements of the given RDD -#' -#' @param X The RDD to apply the transformation. -#' @param FUN the transformation to apply on each element -#' @return a new RDD created by the transformation. -#' @rdname lapply -#' @aliases lapply -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10) -#' multiplyByTwo <- lapply(rdd, function(x) { x * 2 }) -#' collect(multiplyByTwo) # 2,4,6... -#'} +# Apply a function to all elements +# +# This function creates a new RDD by applying the given transformation to all +# elements of the given RDD +# +# @param X The RDD to apply the transformation. +# @param FUN the transformation to apply on each element +# @return a new RDD created by the transformation. +# @rdname lapply +# @aliases lapply +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, 1:10) +# multiplyByTwo <- lapply(rdd, function(x) { x * 2 }) +# collect(multiplyByTwo) # 2,4,6... +#} setMethod("lapply", signature(X = "RDD", FUN = "function"), function(X, FUN) { @@ -463,31 +463,31 @@ setMethod("lapply", lapplyPartitionsWithIndex(X, func) }) -#' @rdname lapply -#' @aliases map,RDD,function-method +# @rdname lapply +# @aliases map,RDD,function-method setMethod("map", signature(X = "RDD", FUN = "function"), function(X, FUN) { lapply(X, FUN) }) -#' Flatten results after apply a function to all elements -#' -#' This function return a new RDD by first applying a function to all -#' elements of this RDD, and then flattening the results. -#' -#' @param X The RDD to apply the transformation. -#' @param FUN the transformation to apply on each element -#' @return a new RDD created by the transformation. -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10) -#' multiplyByTwo <- flatMap(rdd, function(x) { list(x*2, x*10) }) -#' collect(multiplyByTwo) # 2,20,4,40,6,60... -#'} -#' @rdname flatMap -#' @aliases flatMap,RDD,function-method +# Flatten results after apply a function to all elements +# +# This function return a new RDD by first applying a function to all +# elements of this RDD, and then flattening the results. +# +# @param X The RDD to apply the transformation. +# @param FUN the transformation to apply on each element +# @return a new RDD created by the transformation. +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, 1:10) +# multiplyByTwo <- flatMap(rdd, function(x) { list(x*2, x*10) }) +# collect(multiplyByTwo) # 2,20,4,40,6,60... +#} +# @rdname flatMap +# @aliases flatMap,RDD,function-method setMethod("flatMap", signature(X = "RDD", FUN = "function"), function(X, FUN) { @@ -500,83 +500,83 @@ setMethod("flatMap", lapplyPartition(X, partitionFunc) }) -#' Apply a function to each partition of an RDD -#' -#' Return a new RDD by applying a function to each partition of this RDD. -#' -#' @param X The RDD to apply the transformation. -#' @param FUN the transformation to apply on each partition. -#' @return a new RDD created by the transformation. -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10) -#' partitionSum <- lapplyPartition(rdd, function(part) { Reduce("+", part) }) -#' collect(partitionSum) # 15, 40 -#'} -#' @rdname lapplyPartition -#' @aliases lapplyPartition,RDD,function-method +# Apply a function to each partition of an RDD +# +# Return a new RDD by applying a function to each partition of this RDD. +# +# @param X The RDD to apply the transformation. +# @param FUN the transformation to apply on each partition. +# @return a new RDD created by the transformation. +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, 1:10) +# partitionSum <- lapplyPartition(rdd, function(part) { Reduce("+", part) }) +# collect(partitionSum) # 15, 40 +#} +# @rdname lapplyPartition +# @aliases lapplyPartition,RDD,function-method setMethod("lapplyPartition", signature(X = "RDD", FUN = "function"), function(X, FUN) { lapplyPartitionsWithIndex(X, function(s, part) { FUN(part) }) }) -#' mapPartitions is the same as lapplyPartition. -#' -#' @rdname lapplyPartition -#' @aliases mapPartitions,RDD,function-method +# mapPartitions is the same as lapplyPartition. +# +# @rdname lapplyPartition +# @aliases mapPartitions,RDD,function-method setMethod("mapPartitions", signature(X = "RDD", FUN = "function"), function(X, FUN) { lapplyPartition(X, FUN) }) -#' Return a new RDD by applying a function to each partition of this RDD, while -#' tracking the index of the original partition. -#' -#' @param X The RDD to apply the transformation. -#' @param FUN the transformation to apply on each partition; takes the partition -#' index and a list of elements in the particular partition. -#' @return a new RDD created by the transformation. -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10, 5L) -#' prod <- lapplyPartitionsWithIndex(rdd, function(partIndex, part) { -#' partIndex * Reduce("+", part) }) -#' collect(prod, flatten = FALSE) # 0, 7, 22, 45, 76 -#'} -#' @rdname lapplyPartitionsWithIndex -#' @aliases lapplyPartitionsWithIndex,RDD,function-method +# Return a new RDD by applying a function to each partition of this RDD, while +# tracking the index of the original partition. +# +# @param X The RDD to apply the transformation. +# @param FUN the transformation to apply on each partition; takes the partition +# index and a list of elements in the particular partition. +# @return a new RDD created by the transformation. +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, 1:10, 5L) +# prod <- lapplyPartitionsWithIndex(rdd, function(partIndex, part) { +# partIndex * Reduce("+", part) }) +# collect(prod, flatten = FALSE) # 0, 7, 22, 45, 76 +#} +# @rdname lapplyPartitionsWithIndex +# @aliases lapplyPartitionsWithIndex,RDD,function-method setMethod("lapplyPartitionsWithIndex", signature(X = "RDD", FUN = "function"), function(X, FUN) { PipelinedRDD(X, FUN) }) -#' @rdname lapplyPartitionsWithIndex -#' @aliases mapPartitionsWithIndex,RDD,function-method +# @rdname lapplyPartitionsWithIndex +# @aliases mapPartitionsWithIndex,RDD,function-method setMethod("mapPartitionsWithIndex", signature(X = "RDD", FUN = "function"), function(X, FUN) { lapplyPartitionsWithIndex(X, FUN) }) -#' This function returns a new RDD containing only the elements that satisfy -#' a predicate (i.e. returning TRUE in a given logical function). -#' The same as `filter()' in Spark. -#' -#' @param x The RDD to be filtered. -#' @param f A unary predicate function. -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10) -#' unlist(collect(filterRDD(rdd, function (x) { x < 3 }))) # c(1, 2) -#'} -#' @rdname filterRDD -#' @aliases filterRDD,RDD,function-method +# This function returns a new RDD containing only the elements that satisfy +# a predicate (i.e. returning TRUE in a given logical function). +# The same as `filter()' in Spark. +# +# @param x The RDD to be filtered. +# @param f A unary predicate function. +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, 1:10) +# unlist(collect(filterRDD(rdd, function (x) { x < 3 }))) # c(1, 2) +#} +# @rdname filterRDD +# @aliases filterRDD,RDD,function-method setMethod("filterRDD", signature(x = "RDD", f = "function"), function(x, f) { @@ -586,30 +586,30 @@ setMethod("filterRDD", lapplyPartition(x, filter.func) }) -#' @rdname filterRDD -#' @aliases Filter +# @rdname filterRDD +# @aliases Filter setMethod("Filter", signature(f = "function", x = "RDD"), function(f, x) { filterRDD(x, f) }) -#' Reduce across elements of an RDD. -#' -#' This function reduces the elements of this RDD using the -#' specified commutative and associative binary operator. -#' -#' @param x The RDD to reduce -#' @param func Commutative and associative function to apply on elements -#' of the RDD. -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10) -#' reduce(rdd, "+") # 55 -#'} -#' @rdname reduce -#' @aliases reduce,RDD,ANY-method +# Reduce across elements of an RDD. +# +# This function reduces the elements of this RDD using the +# specified commutative and associative binary operator. +# +# @param x The RDD to reduce +# @param func Commutative and associative function to apply on elements +# of the RDD. +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, 1:10) +# reduce(rdd, "+") # 55 +#} +# @rdname reduce +# @aliases reduce,RDD,ANY-method setMethod("reduce", signature(x = "RDD", func = "ANY"), function(x, func) { @@ -623,70 +623,70 @@ setMethod("reduce", Reduce(func, partitionList) }) -#' Get the maximum element of an RDD. -#' -#' @param x The RDD to get the maximum element from -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10) -#' maximum(rdd) # 10 -#'} -#' @rdname maximum -#' @aliases maximum,RDD +# Get the maximum element of an RDD. +# +# @param x The RDD to get the maximum element from +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, 1:10) +# maximum(rdd) # 10 +#} +# @rdname maximum +# @aliases maximum,RDD setMethod("maximum", signature(x = "RDD"), function(x) { reduce(x, max) }) -#' Get the minimum element of an RDD. -#' -#' @param x The RDD to get the minimum element from -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10) -#' minimum(rdd) # 1 -#'} -#' @rdname minimum -#' @aliases minimum,RDD +# Get the minimum element of an RDD. +# +# @param x The RDD to get the minimum element from +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, 1:10) +# minimum(rdd) # 1 +#} +# @rdname minimum +# @aliases minimum,RDD setMethod("minimum", signature(x = "RDD"), function(x) { reduce(x, min) }) -#' Add up the elements in an RDD. -#' -#' @param x The RDD to add up the elements in -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10) -#' sumRDD(rdd) # 55 -#'} -#' @rdname sumRDD -#' @aliases sumRDD,RDD +# Add up the elements in an RDD. +# +# @param x The RDD to add up the elements in +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, 1:10) +# sumRDD(rdd) # 55 +#} +# @rdname sumRDD +# @aliases sumRDD,RDD setMethod("sumRDD", signature(x = "RDD"), function(x) { reduce(x, "+") }) -#' Applies a function to all elements in an RDD, and force evaluation. -#' -#' @param x The RDD to apply the function -#' @param func The function to be applied. -#' @return invisible NULL. -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10) -#' foreach(rdd, function(x) { save(x, file=...) }) -#'} -#' @rdname foreach -#' @aliases foreach,RDD,function-method +# Applies a function to all elements in an RDD, and force evaluation. +# +# @param x The RDD to apply the function +# @param func The function to be applied. +# @return invisible NULL. +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, 1:10) +# foreach(rdd, function(x) { save(x, file=...) }) +#} +# @rdname foreach +# @aliases foreach,RDD,function-method setMethod("foreach", signature(x = "RDD", func = "function"), function(x, func) { @@ -697,37 +697,37 @@ setMethod("foreach", invisible(collect(mapPartitions(x, partition.func))) }) -#' Applies a function to each partition in an RDD, and force evaluation. -#' -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10) -#' foreachPartition(rdd, function(part) { save(part, file=...); NULL }) -#'} -#' @rdname foreach -#' @aliases foreachPartition,RDD,function-method +# Applies a function to each partition in an RDD, and force evaluation. +# +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, 1:10) +# foreachPartition(rdd, function(part) { save(part, file=...); NULL }) +#} +# @rdname foreach +# @aliases foreachPartition,RDD,function-method setMethod("foreachPartition", signature(x = "RDD", func = "function"), function(x, func) { invisible(collect(mapPartitions(x, func))) }) -#' Take elements from an RDD. -#' -#' This function takes the first NUM elements in the RDD and -#' returns them in a list. -#' -#' @param x The RDD to take elements from -#' @param num Number of elements to take -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10) -#' take(rdd, 2L) # list(1, 2) -#'} -#' @rdname take -#' @aliases take,RDD,numeric-method +# Take elements from an RDD. +# +# This function takes the first NUM elements in the RDD and +# returns them in a list. +# +# @param x The RDD to take elements from +# @param num Number of elements to take +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, 1:10) +# take(rdd, 2L) # list(1, 2) +#} +# @rdname take +# @aliases take,RDD,numeric-method setMethod("take", signature(x = "RDD", num = "numeric"), function(x, num) { @@ -762,39 +762,39 @@ setMethod("take", }) -#' First -#' -#' Return the first element of an RDD -#' -#' @rdname first -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10) -#' first(rdd) -#' } +# First +# +# Return the first element of an RDD +# +# @rdname first +# @export +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, 1:10) +# first(rdd) +# } setMethod("first", signature(x = "RDD"), function(x) { take(x, 1)[[1]] }) -#' Removes the duplicates from RDD. -#' -#' This function returns a new RDD containing the distinct elements in the -#' given RDD. The same as `distinct()' in Spark. -#' -#' @param x The RDD to remove duplicates from. -#' @param numPartitions Number of partitions to create. -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, c(1,2,2,3,3,3)) -#' sort(unlist(collect(distinct(rdd)))) # c(1, 2, 3) -#'} -#' @rdname distinct -#' @aliases distinct,RDD-method +# Removes the duplicates from RDD. +# +# This function returns a new RDD containing the distinct elements in the +# given RDD. The same as `distinct()' in Spark. +# +# @param x The RDD to remove duplicates from. +# @param numPartitions Number of partitions to create. +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, c(1,2,2,3,3,3)) +# sort(unlist(collect(distinct(rdd)))) # c(1, 2, 3) +#} +# @rdname distinct +# @aliases distinct,RDD-method setMethod("distinct", signature(x = "RDD"), function(x, numPartitions = SparkR:::numPartitions(x)) { @@ -806,24 +806,24 @@ setMethod("distinct", resRDD }) -#' Return an RDD that is a sampled subset of the given RDD. -#' -#' The same as `sample()' in Spark. (We rename it due to signature -#' inconsistencies with the `sample()' function in R's base package.) -#' -#' @param x The RDD to sample elements from -#' @param withReplacement Sampling with replacement or not -#' @param fraction The (rough) sample target fraction -#' @param seed Randomness seed value -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10) -#' collect(sampleRDD(rdd, FALSE, 0.5, 1618L)) # ~5 distinct elements -#' collect(sampleRDD(rdd, TRUE, 0.5, 9L)) # ~5 elements possibly with duplicates -#'} -#' @rdname sampleRDD -#' @aliases sampleRDD,RDD +# Return an RDD that is a sampled subset of the given RDD. +# +# The same as `sample()' in Spark. (We rename it due to signature +# inconsistencies with the `sample()' function in R's base package.) +# +# @param x The RDD to sample elements from +# @param withReplacement Sampling with replacement or not +# @param fraction The (rough) sample target fraction +# @param seed Randomness seed value +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, 1:10) +# collect(sampleRDD(rdd, FALSE, 0.5, 1618L)) # ~5 distinct elements +# collect(sampleRDD(rdd, TRUE, 0.5, 9L)) # ~5 elements possibly with duplicates +#} +# @rdname sampleRDD +# @aliases sampleRDD,RDD setMethod("sampleRDD", signature(x = "RDD", withReplacement = "logical", fraction = "numeric", seed = "integer"), @@ -867,23 +867,23 @@ setMethod("sampleRDD", lapplyPartitionsWithIndex(x, samplingFunc) }) -#' Return a list of the elements that are a sampled subset of the given RDD. -#' -#' @param x The RDD to sample elements from -#' @param withReplacement Sampling with replacement or not -#' @param num Number of elements to return -#' @param seed Randomness seed value -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:100) -#' # exactly 5 elements sampled, which may not be distinct -#' takeSample(rdd, TRUE, 5L, 1618L) -#' # exactly 5 distinct elements sampled -#' takeSample(rdd, FALSE, 5L, 16181618L) -#'} -#' @rdname takeSample -#' @aliases takeSample,RDD +# Return a list of the elements that are a sampled subset of the given RDD. +# +# @param x The RDD to sample elements from +# @param withReplacement Sampling with replacement or not +# @param num Number of elements to return +# @param seed Randomness seed value +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, 1:100) +# # exactly 5 elements sampled, which may not be distinct +# takeSample(rdd, TRUE, 5L, 1618L) +# # exactly 5 distinct elements sampled +# takeSample(rdd, FALSE, 5L, 16181618L) +#} +# @rdname takeSample +# @aliases takeSample,RDD setMethod("takeSample", signature(x = "RDD", withReplacement = "logical", num = "integer", seed = "integer"), function(x, withReplacement, num, seed) { @@ -930,18 +930,18 @@ setMethod("takeSample", signature(x = "RDD", withReplacement = "logical", sample(samples)[1:total] }) -#' Creates tuples of the elements in this RDD by applying a function. -#' -#' @param x The RDD. -#' @param func The function to be applied. -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list(1, 2, 3)) -#' collect(keyBy(rdd, function(x) { x*x })) # list(list(1, 1), list(4, 2), list(9, 3)) -#'} -#' @rdname keyBy -#' @aliases keyBy,RDD +# Creates tuples of the elements in this RDD by applying a function. +# +# @param x The RDD. +# @param func The function to be applied. +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, list(1, 2, 3)) +# collect(keyBy(rdd, function(x) { x*x })) # list(list(1, 1), list(4, 2), list(9, 3)) +#} +# @rdname keyBy +# @aliases keyBy,RDD setMethod("keyBy", signature(x = "RDD", func = "function"), function(x, func) { @@ -951,44 +951,44 @@ setMethod("keyBy", lapply(x, apply.func) }) -#' Return a new RDD that has exactly numPartitions partitions. -#' Can increase or decrease the level of parallelism in this RDD. Internally, -#' this uses a shuffle to redistribute data. -#' If you are decreasing the number of partitions in this RDD, consider using -#' coalesce, which can avoid performing a shuffle. -#' -#' @param x The RDD. -#' @param numPartitions Number of partitions to create. -#' @seealso coalesce -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list(1, 2, 3, 4, 5, 6, 7), 4L) -#' numPartitions(rdd) # 4 -#' numPartitions(repartition(rdd, 2L)) # 2 -#'} -#' @rdname repartition -#' @aliases repartition,RDD +# Return a new RDD that has exactly numPartitions partitions. +# Can increase or decrease the level of parallelism in this RDD. Internally, +# this uses a shuffle to redistribute data. +# If you are decreasing the number of partitions in this RDD, consider using +# coalesce, which can avoid performing a shuffle. +# +# @param x The RDD. +# @param numPartitions Number of partitions to create. +# @seealso coalesce +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, list(1, 2, 3, 4, 5, 6, 7), 4L) +# numPartitions(rdd) # 4 +# numPartitions(repartition(rdd, 2L)) # 2 +#} +# @rdname repartition +# @aliases repartition,RDD setMethod("repartition", signature(x = "RDD", numPartitions = "numeric"), function(x, numPartitions) { coalesce(x, numPartitions, TRUE) }) -#' Return a new RDD that is reduced into numPartitions partitions. -#' -#' @param x The RDD. -#' @param numPartitions Number of partitions to create. -#' @seealso repartition -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list(1, 2, 3, 4, 5), 3L) -#' numPartitions(rdd) # 3 -#' numPartitions(coalesce(rdd, 1L)) # 1 -#'} -#' @rdname coalesce -#' @aliases coalesce,RDD +# Return a new RDD that is reduced into numPartitions partitions. +# +# @param x The RDD. +# @param numPartitions Number of partitions to create. +# @seealso repartition +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, list(1, 2, 3, 4, 5), 3L) +# numPartitions(rdd) # 3 +# numPartitions(coalesce(rdd, 1L)) # 1 +#} +# @rdname coalesce +# @aliases coalesce,RDD setMethod("coalesce", signature(x = "RDD", numPartitions = "numeric"), function(x, numPartitions, shuffle = FALSE) { @@ -1012,19 +1012,19 @@ setMethod("coalesce", } }) -#' Save this RDD as a SequenceFile of serialized objects. -#' -#' @param x The RDD to save -#' @param path The directory where the file is saved -#' @seealso objectFile -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:3) -#' saveAsObjectFile(rdd, "/tmp/sparkR-tmp") -#'} -#' @rdname saveAsObjectFile -#' @aliases saveAsObjectFile,RDD +# Save this RDD as a SequenceFile of serialized objects. +# +# @param x The RDD to save +# @param path The directory where the file is saved +# @seealso objectFile +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, 1:3) +# saveAsObjectFile(rdd, "/tmp/sparkR-tmp") +#} +# @rdname saveAsObjectFile +# @aliases saveAsObjectFile,RDD setMethod("saveAsObjectFile", signature(x = "RDD", path = "character"), function(x, path) { @@ -1037,18 +1037,18 @@ setMethod("saveAsObjectFile", invisible(callJMethod(getJRDD(x), "saveAsObjectFile", path)) }) -#' Save this RDD as a text file, using string representations of elements. -#' -#' @param x The RDD to save -#' @param path The directory where the partitions of the text file are saved -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:3) -#' saveAsTextFile(rdd, "/tmp/sparkR-tmp") -#'} -#' @rdname saveAsTextFile -#' @aliases saveAsTextFile,RDD +# Save this RDD as a text file, using string representations of elements. +# +# @param x The RDD to save +# @param path The directory where the partitions of the text file are saved +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, 1:3) +# saveAsTextFile(rdd, "/tmp/sparkR-tmp") +#} +# @rdname saveAsTextFile +# @aliases saveAsTextFile,RDD setMethod("saveAsTextFile", signature(x = "RDD", path = "character"), function(x, path) { @@ -1061,21 +1061,21 @@ setMethod("saveAsTextFile", callJMethod(getJRDD(stringRdd, serializedMode = "string"), "saveAsTextFile", path)) }) -#' Sort an RDD by the given key function. -#' -#' @param x An RDD to be sorted. -#' @param func A function used to compute the sort key for each element. -#' @param ascending A flag to indicate whether the sorting is ascending or descending. -#' @param numPartitions Number of partitions to create. -#' @return An RDD where all elements are sorted. -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list(3, 2, 1)) -#' collect(sortBy(rdd, function(x) { x })) # list (1, 2, 3) -#'} -#' @rdname sortBy -#' @aliases sortBy,RDD,RDD-method +# Sort an RDD by the given key function. +# +# @param x An RDD to be sorted. +# @param func A function used to compute the sort key for each element. +# @param ascending A flag to indicate whether the sorting is ascending or descending. +# @param numPartitions Number of partitions to create. +# @return An RDD where all elements are sorted. +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, list(3, 2, 1)) +# collect(sortBy(rdd, function(x) { x })) # list (1, 2, 3) +#} +# @rdname sortBy +# @aliases sortBy,RDD,RDD-method setMethod("sortBy", signature(x = "RDD", func = "function"), function(x, func, ascending = TRUE, numPartitions = SparkR:::numPartitions(x)) { @@ -1137,97 +1137,97 @@ takeOrderedElem <- function(x, num, ascending = TRUE) { resList } -#' Returns the first N elements from an RDD in ascending order. -#' -#' @param x An RDD. -#' @param num Number of elements to return. -#' @return The first N elements from the RDD in ascending order. -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list(10, 1, 2, 9, 3, 4, 5, 6, 7)) -#' takeOrdered(rdd, 6L) # list(1, 2, 3, 4, 5, 6) -#'} -#' @rdname takeOrdered -#' @aliases takeOrdered,RDD,RDD-method +# Returns the first N elements from an RDD in ascending order. +# +# @param x An RDD. +# @param num Number of elements to return. +# @return The first N elements from the RDD in ascending order. +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, list(10, 1, 2, 9, 3, 4, 5, 6, 7)) +# takeOrdered(rdd, 6L) # list(1, 2, 3, 4, 5, 6) +#} +# @rdname takeOrdered +# @aliases takeOrdered,RDD,RDD-method setMethod("takeOrdered", signature(x = "RDD", num = "integer"), function(x, num) { takeOrderedElem(x, num) }) -#' Returns the top N elements from an RDD. -#' -#' @param x An RDD. -#' @param num Number of elements to return. -#' @return The top N elements from the RDD. -#' @rdname top -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list(10, 1, 2, 9, 3, 4, 5, 6, 7)) -#' top(rdd, 6L) # list(10, 9, 7, 6, 5, 4) -#'} -#' @rdname top -#' @aliases top,RDD,RDD-method +# Returns the top N elements from an RDD. +# +# @param x An RDD. +# @param num Number of elements to return. +# @return The top N elements from the RDD. +# @rdname top +# @export +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, list(10, 1, 2, 9, 3, 4, 5, 6, 7)) +# top(rdd, 6L) # list(10, 9, 7, 6, 5, 4) +#} +# @rdname top +# @aliases top,RDD,RDD-method setMethod("top", signature(x = "RDD", num = "integer"), function(x, num) { takeOrderedElem(x, num, FALSE) }) -#' Fold an RDD using a given associative function and a neutral "zero value". -#' -#' Aggregate the elements of each partition, and then the results for all the -#' partitions, using a given associative function and a neutral "zero value". -#' -#' @param x An RDD. -#' @param zeroValue A neutral "zero value". -#' @param op An associative function for the folding operation. -#' @return The folding result. -#' @rdname fold -#' @seealso reduce -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list(1, 2, 3, 4, 5)) -#' fold(rdd, 0, "+") # 15 -#'} -#' @rdname fold -#' @aliases fold,RDD,RDD-method +# Fold an RDD using a given associative function and a neutral "zero value". +# +# Aggregate the elements of each partition, and then the results for all the +# partitions, using a given associative function and a neutral "zero value". +# +# @param x An RDD. +# @param zeroValue A neutral "zero value". +# @param op An associative function for the folding operation. +# @return The folding result. +# @rdname fold +# @seealso reduce +# @export +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, list(1, 2, 3, 4, 5)) +# fold(rdd, 0, "+") # 15 +#} +# @rdname fold +# @aliases fold,RDD,RDD-method setMethod("fold", signature(x = "RDD", zeroValue = "ANY", op = "ANY"), function(x, zeroValue, op) { aggregateRDD(x, zeroValue, op, op) }) -#' Aggregate an RDD using the given combine functions and a neutral "zero value". -#' -#' Aggregate the elements of each partition, and then the results for all the -#' partitions, using given combine functions and a neutral "zero value". -#' -#' @param x An RDD. -#' @param zeroValue A neutral "zero value". -#' @param seqOp A function to aggregate the RDD elements. It may return a different -#' result type from the type of the RDD elements. -#' @param combOp A function to aggregate results of seqOp. -#' @return The aggregation result. -#' @rdname aggregateRDD -#' @seealso reduce -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list(1, 2, 3, 4)) -#' zeroValue <- list(0, 0) -#' seqOp <- function(x, y) { list(x[[1]] + y, x[[2]] + 1) } -#' combOp <- function(x, y) { list(x[[1]] + y[[1]], x[[2]] + y[[2]]) } -#' aggregateRDD(rdd, zeroValue, seqOp, combOp) # list(10, 4) -#'} -#' @rdname aggregateRDD -#' @aliases aggregateRDD,RDD,RDD-method +# Aggregate an RDD using the given combine functions and a neutral "zero value". +# +# Aggregate the elements of each partition, and then the results for all the +# partitions, using given combine functions and a neutral "zero value". +# +# @param x An RDD. +# @param zeroValue A neutral "zero value". +# @param seqOp A function to aggregate the RDD elements. It may return a different +# result type from the type of the RDD elements. +# @param combOp A function to aggregate results of seqOp. +# @return The aggregation result. +# @rdname aggregateRDD +# @seealso reduce +# @export +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, list(1, 2, 3, 4)) +# zeroValue <- list(0, 0) +# seqOp <- function(x, y) { list(x[[1]] + y, x[[2]] + 1) } +# combOp <- function(x, y) { list(x[[1]] + y[[1]], x[[2]] + y[[2]]) } +# aggregateRDD(rdd, zeroValue, seqOp, combOp) # list(10, 4) +#} +# @rdname aggregateRDD +# @aliases aggregateRDD,RDD,RDD-method setMethod("aggregateRDD", signature(x = "RDD", zeroValue = "ANY", seqOp = "ANY", combOp = "ANY"), function(x, zeroValue, seqOp, combOp) { @@ -1240,25 +1240,25 @@ setMethod("aggregateRDD", Reduce(combOp, partitionList, zeroValue) }) -#' Pipes elements to a forked external process. -#' -#' The same as 'pipe()' in Spark. -#' -#' @param x The RDD whose elements are piped to the forked external process. -#' @param command The command to fork an external process. -#' @param env A named list to set environment variables of the external process. -#' @return A new RDD created by piping all elements to a forked external process. -#' @rdname pipeRDD -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10) -#' collect(pipeRDD(rdd, "more") -#' Output: c("1", "2", ..., "10") -#'} -#' @rdname pipeRDD -#' @aliases pipeRDD,RDD,character-method +# Pipes elements to a forked external process. +# +# The same as 'pipe()' in Spark. +# +# @param x The RDD whose elements are piped to the forked external process. +# @param command The command to fork an external process. +# @param env A named list to set environment variables of the external process. +# @return A new RDD created by piping all elements to a forked external process. +# @rdname pipeRDD +# @export +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, 1:10) +# collect(pipeRDD(rdd, "more") +# Output: c("1", "2", ..., "10") +#} +# @rdname pipeRDD +# @aliases pipeRDD,RDD,character-method setMethod("pipeRDD", signature(x = "RDD", command = "character"), function(x, command, env = list()) { @@ -1274,41 +1274,41 @@ setMethod("pipeRDD", }) # TODO: Consider caching the name in the RDD's environment -#' Return an RDD's name. -#' -#' @param x The RDD whose name is returned. -#' @rdname name -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list(1,2,3)) -#' name(rdd) # NULL (if not set before) -#'} -#' @rdname name -#' @aliases name,RDD +# Return an RDD's name. +# +# @param x The RDD whose name is returned. +# @rdname name +# @export +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, list(1,2,3)) +# name(rdd) # NULL (if not set before) +#} +# @rdname name +# @aliases name,RDD setMethod("name", signature(x = "RDD"), function(x) { callJMethod(getJRDD(x), "name") }) -#' Set an RDD's name. -#' -#' @param x The RDD whose name is to be set. -#' @param name The RDD name to be set. -#' @return a new RDD renamed. -#' @rdname setName -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list(1,2,3)) -#' setName(rdd, "myRDD") -#' name(rdd) # "myRDD" -#'} -#' @rdname setName -#' @aliases setName,RDD +# Set an RDD's name. +# +# @param x The RDD whose name is to be set. +# @param name The RDD name to be set. +# @return a new RDD renamed. +# @rdname setName +# @export +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, list(1,2,3)) +# setName(rdd, "myRDD") +# name(rdd) # "myRDD" +#} +# @rdname setName +# @aliases setName,RDD setMethod("setName", signature(x = "RDD", name = "character"), function(x, name) { @@ -1316,25 +1316,25 @@ setMethod("setName", x }) -#' Zip an RDD with generated unique Long IDs. -#' -#' Items in the kth partition will get ids k, n+k, 2*n+k, ..., where -#' n is the number of partitions. So there may exist gaps, but this -#' method won't trigger a spark job, which is different from -#' zipWithIndex. -#' -#' @param x An RDD to be zipped. -#' @return An RDD with zipped items. -#' @seealso zipWithIndex -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list("a", "b", "c", "d", "e"), 3L) -#' collect(zipWithUniqueId(rdd)) -#' # list(list("a", 0), list("b", 3), list("c", 1), list("d", 4), list("e", 2)) -#'} -#' @rdname zipWithUniqueId -#' @aliases zipWithUniqueId,RDD +# Zip an RDD with generated unique Long IDs. +# +# Items in the kth partition will get ids k, n+k, 2*n+k, ..., where +# n is the number of partitions. So there may exist gaps, but this +# method won't trigger a spark job, which is different from +# zipWithIndex. +# +# @param x An RDD to be zipped. +# @return An RDD with zipped items. +# @seealso zipWithIndex +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, list("a", "b", "c", "d", "e"), 3L) +# collect(zipWithUniqueId(rdd)) +# # list(list("a", 0), list("b", 3), list("c", 1), list("d", 4), list("e", 2)) +#} +# @rdname zipWithUniqueId +# @aliases zipWithUniqueId,RDD setMethod("zipWithUniqueId", signature(x = "RDD"), function(x) { @@ -1353,28 +1353,28 @@ setMethod("zipWithUniqueId", lapplyPartitionsWithIndex(x, partitionFunc) }) -#' Zip an RDD with its element indices. -#' -#' The ordering is first based on the partition index and then the -#' ordering of items within each partition. So the first item in -#' the first partition gets index 0, and the last item in the last -#' partition receives the largest index. -#' -#' This method needs to trigger a Spark job when this RDD contains -#' more than one partition. -#' -#' @param x An RDD to be zipped. -#' @return An RDD with zipped items. -#' @seealso zipWithUniqueId -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list("a", "b", "c", "d", "e"), 3L) -#' collect(zipWithIndex(rdd)) -#' # list(list("a", 0), list("b", 1), list("c", 2), list("d", 3), list("e", 4)) -#'} -#' @rdname zipWithIndex -#' @aliases zipWithIndex,RDD +# Zip an RDD with its element indices. +# +# The ordering is first based on the partition index and then the +# ordering of items within each partition. So the first item in +# the first partition gets index 0, and the last item in the last +# partition receives the largest index. +# +# This method needs to trigger a Spark job when this RDD contains +# more than one partition. +# +# @param x An RDD to be zipped. +# @return An RDD with zipped items. +# @seealso zipWithUniqueId +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, list("a", "b", "c", "d", "e"), 3L) +# collect(zipWithIndex(rdd)) +# # list(list("a", 0), list("b", 1), list("c", 2), list("d", 3), list("e", 4)) +#} +# @rdname zipWithIndex +# @aliases zipWithIndex,RDD setMethod("zipWithIndex", signature(x = "RDD"), function(x) { @@ -1406,20 +1406,20 @@ setMethod("zipWithIndex", lapplyPartitionsWithIndex(x, partitionFunc) }) -#' Coalesce all elements within each partition of an RDD into a list. -#' -#' @param x An RDD. -#' @return An RDD created by coalescing all elements within -#' each partition into a list. -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, as.list(1:4), 2L) -#' collect(glom(rdd)) -#' # list(list(1, 2), list(3, 4)) -#'} -#' @rdname glom -#' @aliases glom,RDD +# Coalesce all elements within each partition of an RDD into a list. +# +# @param x An RDD. +# @return An RDD created by coalescing all elements within +# each partition into a list. +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, as.list(1:4), 2L) +# collect(glom(rdd)) +# # list(list(1, 2), list(3, 4)) +#} +# @rdname glom +# @aliases glom,RDD setMethod("glom", signature(x = "RDD"), function(x) { @@ -1432,21 +1432,21 @@ setMethod("glom", ############ Binary Functions ############# -#' Return the union RDD of two RDDs. -#' The same as union() in Spark. -#' -#' @param x An RDD. -#' @param y An RDD. -#' @return a new RDD created by performing the simple union (witout removing -#' duplicates) of two input RDDs. -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:3) -#' unionRDD(rdd, rdd) # 1, 2, 3, 1, 2, 3 -#'} -#' @rdname unionRDD -#' @aliases unionRDD,RDD,RDD-method +# Return the union RDD of two RDDs. +# The same as union() in Spark. +# +# @param x An RDD. +# @param y An RDD. +# @return a new RDD created by performing the simple union (witout removing +# duplicates) of two input RDDs. +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, 1:3) +# unionRDD(rdd, rdd) # 1, 2, 3, 1, 2, 3 +#} +# @rdname unionRDD +# @aliases unionRDD,RDD,RDD-method setMethod("unionRDD", signature(x = "RDD", y = "RDD"), function(x, y) { @@ -1463,27 +1463,27 @@ setMethod("unionRDD", union.rdd }) -#' Zip an RDD with another RDD. -#' -#' Zips this RDD with another one, returning key-value pairs with the -#' first element in each RDD second element in each RDD, etc. Assumes -#' that the two RDDs have the same number of partitions and the same -#' number of elements in each partition (e.g. one was made through -#' a map on the other). -#' -#' @param x An RDD to be zipped. -#' @param other Another RDD to be zipped. -#' @return An RDD zipped from the two RDDs. -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd1 <- parallelize(sc, 0:4) -#' rdd2 <- parallelize(sc, 1000:1004) -#' collect(zipRDD(rdd1, rdd2)) -#' # list(list(0, 1000), list(1, 1001), list(2, 1002), list(3, 1003), list(4, 1004)) -#'} -#' @rdname zipRDD -#' @aliases zipRDD,RDD +# Zip an RDD with another RDD. +# +# Zips this RDD with another one, returning key-value pairs with the +# first element in each RDD second element in each RDD, etc. Assumes +# that the two RDDs have the same number of partitions and the same +# number of elements in each partition (e.g. one was made through +# a map on the other). +# +# @param x An RDD to be zipped. +# @param other Another RDD to be zipped. +# @return An RDD zipped from the two RDDs. +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd1 <- parallelize(sc, 0:4) +# rdd2 <- parallelize(sc, 1000:1004) +# collect(zipRDD(rdd1, rdd2)) +# # list(list(0, 1000), list(1, 1001), list(2, 1002), list(3, 1003), list(4, 1004)) +#} +# @rdname zipRDD +# @aliases zipRDD,RDD setMethod("zipRDD", signature(x = "RDD", other = "RDD"), function(x, other) { @@ -1502,24 +1502,24 @@ setMethod("zipRDD", mergePartitions(rdd, TRUE) }) -#' Cartesian product of this RDD and another one. -#' -#' Return the Cartesian product of this RDD and another one, -#' that is, the RDD of all pairs of elements (a, b) where a -#' is in this and b is in other. -#' -#' @param x An RDD. -#' @param other An RDD. -#' @return A new RDD which is the Cartesian product of these two RDDs. -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:2) -#' sortByKey(cartesian(rdd, rdd)) -#' # list(list(1, 1), list(1, 2), list(2, 1), list(2, 2)) -#'} -#' @rdname cartesian -#' @aliases cartesian,RDD,RDD-method +# Cartesian product of this RDD and another one. +# +# Return the Cartesian product of this RDD and another one, +# that is, the RDD of all pairs of elements (a, b) where a +# is in this and b is in other. +# +# @param x An RDD. +# @param other An RDD. +# @return A new RDD which is the Cartesian product of these two RDDs. +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, 1:2) +# sortByKey(cartesian(rdd, rdd)) +# # list(list(1, 1), list(1, 2), list(2, 1), list(2, 2)) +#} +# @rdname cartesian +# @aliases cartesian,RDD,RDD-method setMethod("cartesian", signature(x = "RDD", other = "RDD"), function(x, other) { @@ -1532,24 +1532,24 @@ setMethod("cartesian", mergePartitions(rdd, FALSE) }) -#' Subtract an RDD with another RDD. -#' -#' Return an RDD with the elements from this that are not in other. -#' -#' @param x An RDD. -#' @param other An RDD. -#' @param numPartitions Number of the partitions in the result RDD. -#' @return An RDD with the elements from this that are not in other. -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd1 <- parallelize(sc, list(1, 1, 2, 2, 3, 4)) -#' rdd2 <- parallelize(sc, list(2, 4)) -#' collect(subtract(rdd1, rdd2)) -#' # list(1, 1, 3) -#'} -#' @rdname subtract -#' @aliases subtract,RDD +# Subtract an RDD with another RDD. +# +# Return an RDD with the elements from this that are not in other. +# +# @param x An RDD. +# @param other An RDD. +# @param numPartitions Number of the partitions in the result RDD. +# @return An RDD with the elements from this that are not in other. +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd1 <- parallelize(sc, list(1, 1, 2, 2, 3, 4)) +# rdd2 <- parallelize(sc, list(2, 4)) +# collect(subtract(rdd1, rdd2)) +# # list(1, 1, 3) +#} +# @rdname subtract +# @aliases subtract,RDD setMethod("subtract", signature(x = "RDD", other = "RDD"), function(x, other, numPartitions = SparkR:::numPartitions(x)) { @@ -1559,28 +1559,28 @@ setMethod("subtract", keys(subtractByKey(rdd1, rdd2, numPartitions)) }) -#' Intersection of this RDD and another one. -#' -#' Return the intersection of this RDD and another one. -#' The output will not contain any duplicate elements, -#' even if the input RDDs did. Performs a hash partition -#' across the cluster. -#' Note that this method performs a shuffle internally. -#' -#' @param x An RDD. -#' @param other An RDD. -#' @param numPartitions The number of partitions in the result RDD. -#' @return An RDD which is the intersection of these two RDDs. -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd1 <- parallelize(sc, list(1, 10, 2, 3, 4, 5)) -#' rdd2 <- parallelize(sc, list(1, 6, 2, 3, 7, 8)) -#' collect(sortBy(intersection(rdd1, rdd2), function(x) { x })) -#' # list(1, 2, 3) -#'} -#' @rdname intersection -#' @aliases intersection,RDD +# Intersection of this RDD and another one. +# +# Return the intersection of this RDD and another one. +# The output will not contain any duplicate elements, +# even if the input RDDs did. Performs a hash partition +# across the cluster. +# Note that this method performs a shuffle internally. +# +# @param x An RDD. +# @param other An RDD. +# @param numPartitions The number of partitions in the result RDD. +# @return An RDD which is the intersection of these two RDDs. +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd1 <- parallelize(sc, list(1, 10, 2, 3, 4, 5)) +# rdd2 <- parallelize(sc, list(1, 6, 2, 3, 7, 8)) +# collect(sortBy(intersection(rdd1, rdd2), function(x) { x })) +# # list(1, 2, 3) +#} +# @rdname intersection +# @aliases intersection,RDD setMethod("intersection", signature(x = "RDD", other = "RDD"), function(x, other, numPartitions = SparkR:::numPartitions(x)) { @@ -1596,26 +1596,26 @@ setMethod("intersection", keys(filterRDD(cogroup(rdd1, rdd2, numPartitions = numPartitions), filterFunction)) }) -#' Zips an RDD's partitions with one (or more) RDD(s). -#' Same as zipPartitions in Spark. -#' -#' @param ... RDDs to be zipped. -#' @param func A function to transform zipped partitions. -#' @return A new RDD by applying a function to the zipped partitions. -#' Assumes that all the RDDs have the *same number of partitions*, but -#' does *not* require them to have the same number of elements in each partition. -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd1 <- parallelize(sc, 1:2, 2L) # 1, 2 -#' rdd2 <- parallelize(sc, 1:4, 2L) # 1:2, 3:4 -#' rdd3 <- parallelize(sc, 1:6, 2L) # 1:3, 4:6 -#' collect(zipPartitions(rdd1, rdd2, rdd3, -#' func = function(x, y, z) { list(list(x, y, z))} )) -#' # list(list(1, c(1,2), c(1,2,3)), list(2, c(3,4), c(4,5,6))) -#'} -#' @rdname zipRDD -#' @aliases zipPartitions,RDD +# Zips an RDD's partitions with one (or more) RDD(s). +# Same as zipPartitions in Spark. +# +# @param ... RDDs to be zipped. +# @param func A function to transform zipped partitions. +# @return A new RDD by applying a function to the zipped partitions. +# Assumes that all the RDDs have the *same number of partitions*, but +# does *not* require them to have the same number of elements in each partition. +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd1 <- parallelize(sc, 1:2, 2L) # 1, 2 +# rdd2 <- parallelize(sc, 1:4, 2L) # 1:2, 3:4 +# rdd3 <- parallelize(sc, 1:6, 2L) # 1:3, 4:6 +# collect(zipPartitions(rdd1, rdd2, rdd3, +# func = function(x, y, z) { list(list(x, y, z))} )) +# # list(list(1, c(1,2), c(1,2,3)), list(2, c(3,4), c(4,5,6))) +#} +# @rdname zipRDD +# @aliases zipPartitions,RDD setMethod("zipPartitions", "RDD", function(..., func) { diff --git a/R/pkg/R/SQLContext.R b/R/pkg/R/SQLContext.R index 4f05ba524a01a..cae06e6af2bff 100644 --- a/R/pkg/R/SQLContext.R +++ b/R/pkg/R/SQLContext.R @@ -150,21 +150,21 @@ createDataFrame <- function(sqlCtx, data, schema = NULL, samplingRatio = 1.0) { dataFrame(sdf) } -#' toDF -#' -#' Converts an RDD to a DataFrame by infer the types. -#' -#' @param x An RDD -#' -#' @rdname DataFrame -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' sqlCtx <- sparkRSQL.init(sc) -#' rdd <- lapply(parallelize(sc, 1:10), function(x) list(a=x, b=as.character(x))) -#' df <- toDF(rdd) -#' } +# toDF +# +# Converts an RDD to a DataFrame by infer the types. +# +# @param x An RDD +# +# @rdname DataFrame +# @export +# @examples +#\dontrun{ +# sc <- sparkR.init() +# sqlCtx <- sparkRSQL.init(sc) +# rdd <- lapply(parallelize(sc, 1:10), function(x) list(a=x, b=as.character(x))) +# df <- toDF(rdd) +# } setGeneric("toDF", function(x, ...) { standardGeneric("toDF") }) @@ -207,23 +207,23 @@ jsonFile <- function(sqlCtx, path) { } -#' JSON RDD -#' -#' Loads an RDD storing one JSON object per string as a DataFrame. -#' -#' @param sqlCtx SQLContext to use -#' @param rdd An RDD of JSON string -#' @param schema A StructType object to use as schema -#' @param samplingRatio The ratio of simpling used to infer the schema -#' @return A DataFrame -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' sqlCtx <- sparkRSQL.init(sc) -#' rdd <- texFile(sc, "path/to/json") -#' df <- jsonRDD(sqlCtx, rdd) -#' } +# JSON RDD +# +# Loads an RDD storing one JSON object per string as a DataFrame. +# +# @param sqlCtx SQLContext to use +# @param rdd An RDD of JSON string +# @param schema A StructType object to use as schema +# @param samplingRatio The ratio of simpling used to infer the schema +# @return A DataFrame +# @export +# @examples +#\dontrun{ +# sc <- sparkR.init() +# sqlCtx <- sparkRSQL.init(sc) +# rdd <- texFile(sc, "path/to/json") +# df <- jsonRDD(sqlCtx, rdd) +# } # TODO: support schema jsonRDD <- function(sqlCtx, rdd, schema = NULL, samplingRatio = 1.0) { diff --git a/R/pkg/R/broadcast.R b/R/pkg/R/broadcast.R index 583fa2e7fdcfd..23dc38780716e 100644 --- a/R/pkg/R/broadcast.R +++ b/R/pkg/R/broadcast.R @@ -23,21 +23,21 @@ .broadcastValues <- new.env() .broadcastIdToName <- new.env() -#' @title S4 class that represents a Broadcast variable -#' @description Broadcast variables can be created using the broadcast -#' function from a \code{SparkContext}. -#' @rdname broadcast-class -#' @seealso broadcast -#' -#' @param id Id of the backing Spark broadcast variable -#' @export +# @title S4 class that represents a Broadcast variable +# @description Broadcast variables can be created using the broadcast +# function from a \code{SparkContext}. +# @rdname broadcast-class +# @seealso broadcast +# +# @param id Id of the backing Spark broadcast variable +# @export setClass("Broadcast", slots = list(id = "character")) -#' @rdname broadcast-class -#' @param value Value of the broadcast variable -#' @param jBroadcastRef reference to the backing Java broadcast object -#' @param objName name of broadcasted object -#' @export +# @rdname broadcast-class +# @param value Value of the broadcast variable +# @param jBroadcastRef reference to the backing Java broadcast object +# @param objName name of broadcasted object +# @export Broadcast <- function(id, value, jBroadcastRef, objName) { .broadcastValues[[id]] <- value .broadcastNames[[as.character(objName)]] <- jBroadcastRef @@ -45,13 +45,13 @@ Broadcast <- function(id, value, jBroadcastRef, objName) { new("Broadcast", id = id) } -#' @description -#' \code{value} can be used to get the value of a broadcast variable inside -#' a distributed function. -#' -#' @param bcast The broadcast variable to get -#' @rdname broadcast -#' @aliases value,Broadcast-method +# @description +# \code{value} can be used to get the value of a broadcast variable inside +# a distributed function. +# +# @param bcast The broadcast variable to get +# @rdname broadcast +# @aliases value,Broadcast-method setMethod("value", signature(bcast = "Broadcast"), function(bcast) { @@ -62,24 +62,24 @@ setMethod("value", } }) -#' Internal function to set values of a broadcast variable. -#' -#' This function is used internally by Spark to set the value of a broadcast -#' variable on workers. Not intended for use outside the package. -#' -#' @rdname broadcast-internal -#' @seealso broadcast, value +# Internal function to set values of a broadcast variable. +# +# This function is used internally by Spark to set the value of a broadcast +# variable on workers. Not intended for use outside the package. +# +# @rdname broadcast-internal +# @seealso broadcast, value -#' @param bcastId The id of broadcast variable to set -#' @param value The value to be set -#' @export +# @param bcastId The id of broadcast variable to set +# @param value The value to be set +# @export setBroadcastValue <- function(bcastId, value) { bcastIdStr <- as.character(bcastId) .broadcastValues[[bcastIdStr]] <- value } -#' Helper function to clear the list of broadcast variables we know about -#' Should be called when the SparkR JVM backend is shutdown +# Helper function to clear the list of broadcast variables we know about +# Should be called when the SparkR JVM backend is shutdown clearBroadcastVariables <- function() { bcasts <- ls(.broadcastNames) rm(list = bcasts, envir = .broadcastNames) diff --git a/R/pkg/R/context.R b/R/pkg/R/context.R index b4845b6948997..43be9c904fdf6 100644 --- a/R/pkg/R/context.R +++ b/R/pkg/R/context.R @@ -25,27 +25,27 @@ getMinPartitions <- function(sc, minPartitions) { as.integer(minPartitions) } -#' Create an RDD from a text file. -#' -#' This function reads a text file from HDFS, a local file system (available on all -#' nodes), or any Hadoop-supported file system URI, and creates an -#' RDD of strings from it. -#' -#' @param sc SparkContext to use -#' @param path Path of file to read. A vector of multiple paths is allowed. -#' @param minPartitions Minimum number of partitions to be created. If NULL, the default -#' value is chosen based on available parallelism. -#' @return RDD where each item is of type \code{character} -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' lines <- textFile(sc, "myfile.txt") -#'} +# Create an RDD from a text file. +# +# This function reads a text file from HDFS, a local file system (available on all +# nodes), or any Hadoop-supported file system URI, and creates an +# RDD of strings from it. +# +# @param sc SparkContext to use +# @param path Path of file to read. A vector of multiple paths is allowed. +# @param minPartitions Minimum number of partitions to be created. If NULL, the default +# value is chosen based on available parallelism. +# @return RDD where each item is of type \code{character} +# @export +# @examples +#\dontrun{ +# sc <- sparkR.init() +# lines <- textFile(sc, "myfile.txt") +#} textFile <- function(sc, path, minPartitions = NULL) { # Allow the user to have a more flexible definiton of the text file path path <- suppressWarnings(normalizePath(path)) - #' Convert a string vector of paths to a string containing comma separated paths + # Convert a string vector of paths to a string containing comma separated paths path <- paste(path, collapse = ",") jrdd <- callJMethod(sc, "textFile", path, getMinPartitions(sc, minPartitions)) @@ -53,27 +53,27 @@ textFile <- function(sc, path, minPartitions = NULL) { RDD(jrdd, "string") } -#' Load an RDD saved as a SequenceFile containing serialized objects. -#' -#' The file to be loaded should be one that was previously generated by calling -#' saveAsObjectFile() of the RDD class. -#' -#' @param sc SparkContext to use -#' @param path Path of file to read. A vector of multiple paths is allowed. -#' @param minPartitions Minimum number of partitions to be created. If NULL, the default -#' value is chosen based on available parallelism. -#' @return RDD containing serialized R objects. -#' @seealso saveAsObjectFile -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- objectFile(sc, "myfile") -#'} +# Load an RDD saved as a SequenceFile containing serialized objects. +# +# The file to be loaded should be one that was previously generated by calling +# saveAsObjectFile() of the RDD class. +# +# @param sc SparkContext to use +# @param path Path of file to read. A vector of multiple paths is allowed. +# @param minPartitions Minimum number of partitions to be created. If NULL, the default +# value is chosen based on available parallelism. +# @return RDD containing serialized R objects. +# @seealso saveAsObjectFile +# @export +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- objectFile(sc, "myfile") +#} objectFile <- function(sc, path, minPartitions = NULL) { # Allow the user to have a more flexible definiton of the text file path path <- suppressWarnings(normalizePath(path)) - #' Convert a string vector of paths to a string containing comma separated paths + # Convert a string vector of paths to a string containing comma separated paths path <- paste(path, collapse = ",") jrdd <- callJMethod(sc, "objectFile", path, getMinPartitions(sc, minPartitions)) @@ -81,24 +81,24 @@ objectFile <- function(sc, path, minPartitions = NULL) { RDD(jrdd, "byte") } -#' Create an RDD from a homogeneous list or vector. -#' -#' This function creates an RDD from a local homogeneous list in R. The elements -#' in the list are split into \code{numSlices} slices and distributed to nodes -#' in the cluster. -#' -#' @param sc SparkContext to use -#' @param coll collection to parallelize -#' @param numSlices number of partitions to create in the RDD -#' @return an RDD created from this collection -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10, 2) -#' # The RDD should contain 10 elements -#' length(rdd) -#'} +# Create an RDD from a homogeneous list or vector. +# +# This function creates an RDD from a local homogeneous list in R. The elements +# in the list are split into \code{numSlices} slices and distributed to nodes +# in the cluster. +# +# @param sc SparkContext to use +# @param coll collection to parallelize +# @param numSlices number of partitions to create in the RDD +# @return an RDD created from this collection +# @export +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, 1:10, 2) +# # The RDD should contain 10 elements +# length(rdd) +#} parallelize <- function(sc, coll, numSlices = 1) { # TODO: bound/safeguard numSlices # TODO: unit tests for if the split works for all primitives @@ -133,33 +133,33 @@ parallelize <- function(sc, coll, numSlices = 1) { RDD(jrdd, "byte") } -#' Include this specified package on all workers -#' -#' This function can be used to include a package on all workers before the -#' user's code is executed. This is useful in scenarios where other R package -#' functions are used in a function passed to functions like \code{lapply}. -#' NOTE: The package is assumed to be installed on every node in the Spark -#' cluster. -#' -#' @param sc SparkContext to use -#' @param pkg Package name -#' -#' @export -#' @examples -#'\dontrun{ -#' library(Matrix) -#' -#' sc <- sparkR.init() -#' # Include the matrix library we will be using -#' includePackage(sc, Matrix) -#' -#' generateSparse <- function(x) { -#' sparseMatrix(i=c(1, 2, 3), j=c(1, 2, 3), x=c(1, 2, 3)) -#' } -#' -#' rdd <- lapplyPartition(parallelize(sc, 1:2, 2L), generateSparse) -#' collect(rdd) -#'} +# Include this specified package on all workers +# +# This function can be used to include a package on all workers before the +# user's code is executed. This is useful in scenarios where other R package +# functions are used in a function passed to functions like \code{lapply}. +# NOTE: The package is assumed to be installed on every node in the Spark +# cluster. +# +# @param sc SparkContext to use +# @param pkg Package name +# +# @export +# @examples +#\dontrun{ +# library(Matrix) +# +# sc <- sparkR.init() +# # Include the matrix library we will be using +# includePackage(sc, Matrix) +# +# generateSparse <- function(x) { +# sparseMatrix(i=c(1, 2, 3), j=c(1, 2, 3), x=c(1, 2, 3)) +# } +# +# rdd <- lapplyPartition(parallelize(sc, 1:2, 2L), generateSparse) +# collect(rdd) +#} includePackage <- function(sc, pkg) { pkg <- as.character(substitute(pkg)) if (exists(".packages", .sparkREnv)) { @@ -171,30 +171,30 @@ includePackage <- function(sc, pkg) { .sparkREnv$.packages <- packages } -#' @title Broadcast a variable to all workers -#' -#' @description -#' Broadcast a read-only variable to the cluster, returning a \code{Broadcast} -#' object for reading it in distributed functions. -#' -#' @param sc Spark Context to use -#' @param object Object to be broadcast -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:2, 2L) -#' -#' # Large Matrix object that we want to broadcast -#' randomMat <- matrix(nrow=100, ncol=10, data=rnorm(1000)) -#' randomMatBr <- broadcast(sc, randomMat) -#' -#' # Use the broadcast variable inside the function -#' useBroadcast <- function(x) { -#' sum(value(randomMatBr) * x) -#' } -#' sumRDD <- lapply(rdd, useBroadcast) -#'} +# @title Broadcast a variable to all workers +# +# @description +# Broadcast a read-only variable to the cluster, returning a \code{Broadcast} +# object for reading it in distributed functions. +# +# @param sc Spark Context to use +# @param object Object to be broadcast +# @export +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, 1:2, 2L) +# +# # Large Matrix object that we want to broadcast +# randomMat <- matrix(nrow=100, ncol=10, data=rnorm(1000)) +# randomMatBr <- broadcast(sc, randomMat) +# +# # Use the broadcast variable inside the function +# useBroadcast <- function(x) { +# sum(value(randomMatBr) * x) +# } +# sumRDD <- lapply(rdd, useBroadcast) +#} broadcast <- function(sc, object) { objName <- as.character(substitute(object)) serializedObj <- serialize(object, connection = NULL) @@ -205,21 +205,21 @@ broadcast <- function(sc, object) { Broadcast(id, object, jBroadcast, objName) } -#' @title Set the checkpoint directory -#' -#' Set the directory under which RDDs are going to be checkpointed. The -#' directory must be a HDFS path if running on a cluster. -#' -#' @param sc Spark Context to use -#' @param dirName Directory path -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' setCheckpointDir(sc, "~/checkpoint") -#' rdd <- parallelize(sc, 1:2, 2L) -#' checkpoint(rdd) -#'} +# @title Set the checkpoint directory +# +# Set the directory under which RDDs are going to be checkpointed. The +# directory must be a HDFS path if running on a cluster. +# +# @param sc Spark Context to use +# @param dirName Directory path +# @export +# @examples +#\dontrun{ +# sc <- sparkR.init() +# setCheckpointDir(sc, "~/checkpoint") +# rdd <- parallelize(sc, 1:2, 2L) +# checkpoint(rdd) +#} setCheckpointDir <- function(sc, dirName) { invisible(callJMethod(sc, "setCheckpointDir", suppressWarnings(normalizePath(dirName)))) } diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index 5838955f74dad..380e8ebe8c8f4 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -17,353 +17,353 @@ ############ RDD Actions and Transformations ############ -#' @rdname aggregateRDD -#' @seealso reduce -#' @export +# @rdname aggregateRDD +# @seealso reduce +# @export setGeneric("aggregateRDD", function(x, zeroValue, seqOp, combOp) { standardGeneric("aggregateRDD") }) -#' @rdname cache-methods -#' @export +# @rdname cache-methods +# @export setGeneric("cache", function(x) { standardGeneric("cache") }) -#' @rdname coalesce -#' @seealso repartition -#' @export +# @rdname coalesce +# @seealso repartition +# @export setGeneric("coalesce", function(x, numPartitions, ...) { standardGeneric("coalesce") }) -#' @rdname checkpoint-methods -#' @export +# @rdname checkpoint-methods +# @export setGeneric("checkpoint", function(x) { standardGeneric("checkpoint") }) -#' @rdname collect-methods -#' @export +# @rdname collect-methods +# @export setGeneric("collect", function(x, ...) { standardGeneric("collect") }) -#' @rdname collect-methods -#' @export +# @rdname collect-methods +# @export setGeneric("collectAsMap", function(x) { standardGeneric("collectAsMap") }) -#' @rdname collect-methods -#' @export +# @rdname collect-methods +# @export setGeneric("collectPartition", function(x, partitionId) { standardGeneric("collectPartition") }) -#' @rdname count -#' @export +# @rdname count +# @export setGeneric("count", function(x) { standardGeneric("count") }) -#' @rdname countByValue -#' @export +# @rdname countByValue +# @export setGeneric("countByValue", function(x) { standardGeneric("countByValue") }) -#' @rdname distinct -#' @export +# @rdname distinct +# @export setGeneric("distinct", function(x, numPartitions = 1) { standardGeneric("distinct") }) -#' @rdname filterRDD -#' @export +# @rdname filterRDD +# @export setGeneric("filterRDD", function(x, f) { standardGeneric("filterRDD") }) -#' @rdname first -#' @export +# @rdname first +# @export setGeneric("first", function(x) { standardGeneric("first") }) -#' @rdname flatMap -#' @export +# @rdname flatMap +# @export setGeneric("flatMap", function(X, FUN) { standardGeneric("flatMap") }) -#' @rdname fold -#' @seealso reduce -#' @export +# @rdname fold +# @seealso reduce +# @export setGeneric("fold", function(x, zeroValue, op) { standardGeneric("fold") }) -#' @rdname foreach -#' @export +# @rdname foreach +# @export setGeneric("foreach", function(x, func) { standardGeneric("foreach") }) -#' @rdname foreach -#' @export +# @rdname foreach +# @export setGeneric("foreachPartition", function(x, func) { standardGeneric("foreachPartition") }) # The jrdd accessor function. setGeneric("getJRDD", function(rdd, ...) { standardGeneric("getJRDD") }) -#' @rdname glom -#' @export +# @rdname glom +# @export setGeneric("glom", function(x) { standardGeneric("glom") }) -#' @rdname keyBy -#' @export +# @rdname keyBy +# @export setGeneric("keyBy", function(x, func) { standardGeneric("keyBy") }) -#' @rdname lapplyPartition -#' @export +# @rdname lapplyPartition +# @export setGeneric("lapplyPartition", function(X, FUN) { standardGeneric("lapplyPartition") }) -#' @rdname lapplyPartitionsWithIndex -#' @export +# @rdname lapplyPartitionsWithIndex +# @export setGeneric("lapplyPartitionsWithIndex", function(X, FUN) { standardGeneric("lapplyPartitionsWithIndex") }) -#' @rdname lapply -#' @export +# @rdname lapply +# @export setGeneric("map", function(X, FUN) { standardGeneric("map") }) -#' @rdname lapplyPartition -#' @export +# @rdname lapplyPartition +# @export setGeneric("mapPartitions", function(X, FUN) { standardGeneric("mapPartitions") }) -#' @rdname lapplyPartitionsWithIndex -#' @export +# @rdname lapplyPartitionsWithIndex +# @export setGeneric("mapPartitionsWithIndex", function(X, FUN) { standardGeneric("mapPartitionsWithIndex") }) -#' @rdname maximum -#' @export +# @rdname maximum +# @export setGeneric("maximum", function(x) { standardGeneric("maximum") }) -#' @rdname minimum -#' @export +# @rdname minimum +# @export setGeneric("minimum", function(x) { standardGeneric("minimum") }) -#' @rdname sumRDD -#' @export +# @rdname sumRDD +# @export setGeneric("sumRDD", function(x) { standardGeneric("sumRDD") }) -#' @rdname name -#' @export +# @rdname name +# @export setGeneric("name", function(x) { standardGeneric("name") }) -#' @rdname numPartitions -#' @export +# @rdname numPartitions +# @export setGeneric("numPartitions", function(x) { standardGeneric("numPartitions") }) -#' @rdname persist -#' @export +# @rdname persist +# @export setGeneric("persist", function(x, newLevel) { standardGeneric("persist") }) -#' @rdname pipeRDD -#' @export +# @rdname pipeRDD +# @export setGeneric("pipeRDD", function(x, command, env = list()) { standardGeneric("pipeRDD")}) -#' @rdname reduce -#' @export +# @rdname reduce +# @export setGeneric("reduce", function(x, func) { standardGeneric("reduce") }) -#' @rdname repartition -#' @seealso coalesce -#' @export +# @rdname repartition +# @seealso coalesce +# @export setGeneric("repartition", function(x, numPartitions) { standardGeneric("repartition") }) -#' @rdname sampleRDD -#' @export +# @rdname sampleRDD +# @export setGeneric("sampleRDD", function(x, withReplacement, fraction, seed) { standardGeneric("sampleRDD") }) -#' @rdname saveAsObjectFile -#' @seealso objectFile -#' @export +# @rdname saveAsObjectFile +# @seealso objectFile +# @export setGeneric("saveAsObjectFile", function(x, path) { standardGeneric("saveAsObjectFile") }) -#' @rdname saveAsTextFile -#' @export +# @rdname saveAsTextFile +# @export setGeneric("saveAsTextFile", function(x, path) { standardGeneric("saveAsTextFile") }) -#' @rdname setName -#' @export +# @rdname setName +# @export setGeneric("setName", function(x, name) { standardGeneric("setName") }) -#' @rdname sortBy -#' @export +# @rdname sortBy +# @export setGeneric("sortBy", function(x, func, ascending = TRUE, numPartitions = 1) { standardGeneric("sortBy") }) -#' @rdname take -#' @export +# @rdname take +# @export setGeneric("take", function(x, num) { standardGeneric("take") }) -#' @rdname takeOrdered -#' @export +# @rdname takeOrdered +# @export setGeneric("takeOrdered", function(x, num) { standardGeneric("takeOrdered") }) -#' @rdname takeSample -#' @export +# @rdname takeSample +# @export setGeneric("takeSample", function(x, withReplacement, num, seed) { standardGeneric("takeSample") }) -#' @rdname top -#' @export +# @rdname top +# @export setGeneric("top", function(x, num) { standardGeneric("top") }) -#' @rdname unionRDD -#' @export +# @rdname unionRDD +# @export setGeneric("unionRDD", function(x, y) { standardGeneric("unionRDD") }) -#' @rdname unpersist-methods -#' @export +# @rdname unpersist-methods +# @export setGeneric("unpersist", function(x, ...) { standardGeneric("unpersist") }) -#' @rdname zipRDD -#' @export +# @rdname zipRDD +# @export setGeneric("zipRDD", function(x, other) { standardGeneric("zipRDD") }) -#' @rdname zipRDD -#' @export +# @rdname zipRDD +# @export setGeneric("zipPartitions", function(..., func) { standardGeneric("zipPartitions") }, signature = "...") -#' @rdname zipWithIndex -#' @seealso zipWithUniqueId -#' @export +# @rdname zipWithIndex +# @seealso zipWithUniqueId +# @export setGeneric("zipWithIndex", function(x) { standardGeneric("zipWithIndex") }) -#' @rdname zipWithUniqueId -#' @seealso zipWithIndex -#' @export +# @rdname zipWithUniqueId +# @seealso zipWithIndex +# @export setGeneric("zipWithUniqueId", function(x) { standardGeneric("zipWithUniqueId") }) ############ Binary Functions ############# -#' @rdname cartesian -#' @export +# @rdname cartesian +# @export setGeneric("cartesian", function(x, other) { standardGeneric("cartesian") }) -#' @rdname countByKey -#' @export +# @rdname countByKey +# @export setGeneric("countByKey", function(x) { standardGeneric("countByKey") }) -#' @rdname flatMapValues -#' @export +# @rdname flatMapValues +# @export setGeneric("flatMapValues", function(X, FUN) { standardGeneric("flatMapValues") }) -#' @rdname intersection -#' @export +# @rdname intersection +# @export setGeneric("intersection", function(x, other, numPartitions = 1) { standardGeneric("intersection") }) -#' @rdname keys -#' @export +# @rdname keys +# @export setGeneric("keys", function(x) { standardGeneric("keys") }) -#' @rdname lookup -#' @export +# @rdname lookup +# @export setGeneric("lookup", function(x, key) { standardGeneric("lookup") }) -#' @rdname mapValues -#' @export +# @rdname mapValues +# @export setGeneric("mapValues", function(X, FUN) { standardGeneric("mapValues") }) -#' @rdname sampleByKey -#' @export +# @rdname sampleByKey +# @export setGeneric("sampleByKey", function(x, withReplacement, fractions, seed) { standardGeneric("sampleByKey") }) -#' @rdname values -#' @export +# @rdname values +# @export setGeneric("values", function(x) { standardGeneric("values") }) ############ Shuffle Functions ############ -#' @rdname aggregateByKey -#' @seealso foldByKey, combineByKey -#' @export +# @rdname aggregateByKey +# @seealso foldByKey, combineByKey +# @export setGeneric("aggregateByKey", function(x, zeroValue, seqOp, combOp, numPartitions) { standardGeneric("aggregateByKey") }) -#' @rdname cogroup -#' @export +# @rdname cogroup +# @export setGeneric("cogroup", function(..., numPartitions) { standardGeneric("cogroup") }, signature = "...") -#' @rdname combineByKey -#' @seealso groupByKey, reduceByKey -#' @export +# @rdname combineByKey +# @seealso groupByKey, reduceByKey +# @export setGeneric("combineByKey", function(x, createCombiner, mergeValue, mergeCombiners, numPartitions) { standardGeneric("combineByKey") }) -#' @rdname foldByKey -#' @seealso aggregateByKey, combineByKey -#' @export +# @rdname foldByKey +# @seealso aggregateByKey, combineByKey +# @export setGeneric("foldByKey", function(x, zeroValue, func, numPartitions) { standardGeneric("foldByKey") }) -#' @rdname join-methods -#' @export +# @rdname join-methods +# @export setGeneric("fullOuterJoin", function(x, y, numPartitions) { standardGeneric("fullOuterJoin") }) -#' @rdname groupByKey -#' @seealso reduceByKey -#' @export +# @rdname groupByKey +# @seealso reduceByKey +# @export setGeneric("groupByKey", function(x, numPartitions) { standardGeneric("groupByKey") }) -#' @rdname join-methods -#' @export +# @rdname join-methods +# @export setGeneric("join", function(x, y, ...) { standardGeneric("join") }) -#' @rdname join-methods -#' @export +# @rdname join-methods +# @export setGeneric("leftOuterJoin", function(x, y, numPartitions) { standardGeneric("leftOuterJoin") }) -#' @rdname partitionBy -#' @export +# @rdname partitionBy +# @export setGeneric("partitionBy", function(x, numPartitions, ...) { standardGeneric("partitionBy") }) -#' @rdname reduceByKey -#' @seealso groupByKey -#' @export +# @rdname reduceByKey +# @seealso groupByKey +# @export setGeneric("reduceByKey", function(x, combineFunc, numPartitions) { standardGeneric("reduceByKey")}) -#' @rdname reduceByKeyLocally -#' @seealso reduceByKey -#' @export +# @rdname reduceByKeyLocally +# @seealso reduceByKey +# @export setGeneric("reduceByKeyLocally", function(x, combineFunc) { standardGeneric("reduceByKeyLocally") }) -#' @rdname join-methods -#' @export +# @rdname join-methods +# @export setGeneric("rightOuterJoin", function(x, y, numPartitions) { standardGeneric("rightOuterJoin") }) -#' @rdname sortByKey -#' @export +# @rdname sortByKey +# @export setGeneric("sortByKey", function(x, ascending = TRUE, numPartitions = 1) { standardGeneric("sortByKey") }) -#' @rdname subtract -#' @export +# @rdname subtract +# @export setGeneric("subtract", function(x, other, numPartitions = 1) { standardGeneric("subtract") }) -#' @rdname subtractByKey -#' @export +# @rdname subtractByKey +# @export setGeneric("subtractByKey", function(x, other, numPartitions = 1) { standardGeneric("subtractByKey") @@ -372,8 +372,8 @@ setGeneric("subtractByKey", ################### Broadcast Variable Methods ################# -#' @rdname broadcast -#' @export +# @rdname broadcast +# @export setGeneric("value", function(bcast) { standardGeneric("value") }) @@ -477,8 +477,8 @@ setGeneric("showDF", function(x,...) { standardGeneric("showDF") }) #' @export setGeneric("sortDF", function(x, col, ...) { standardGeneric("sortDF") }) -#' @rdname tojson -#' @export +# @rdname tojson +# @export setGeneric("toJSON", function(x) { standardGeneric("toJSON") }) #' @rdname DataFrame diff --git a/R/pkg/R/pairRDD.R b/R/pkg/R/pairRDD.R index edeb8d9f75f7b..7694652856da5 100644 --- a/R/pkg/R/pairRDD.R +++ b/R/pkg/R/pairRDD.R @@ -21,23 +21,23 @@ NULL ############ Actions and Transformations ############ -#' Look up elements of a key in an RDD -#' -#' @description -#' \code{lookup} returns a list of values in this RDD for key key. -#' -#' @param x The RDD to collect -#' @param key The key to look up for -#' @return a list of values in this RDD for key key -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' pairs <- list(c(1, 1), c(2, 2), c(1, 3)) -#' rdd <- parallelize(sc, pairs) -#' lookup(rdd, 1) # list(1, 3) -#'} -#' @rdname lookup -#' @aliases lookup,RDD-method +# Look up elements of a key in an RDD +# +# @description +# \code{lookup} returns a list of values in this RDD for key key. +# +# @param x The RDD to collect +# @param key The key to look up for +# @return a list of values in this RDD for key key +# @examples +#\dontrun{ +# sc <- sparkR.init() +# pairs <- list(c(1, 1), c(2, 2), c(1, 3)) +# rdd <- parallelize(sc, pairs) +# lookup(rdd, 1) # list(1, 3) +#} +# @rdname lookup +# @aliases lookup,RDD-method setMethod("lookup", signature(x = "RDD", key = "ANY"), function(x, key) { @@ -49,21 +49,21 @@ setMethod("lookup", collect(valsRDD) }) -#' Count the number of elements for each key, and return the result to the -#' master as lists of (key, count) pairs. -#' -#' Same as countByKey in Spark. -#' -#' @param x The RDD to count keys. -#' @return list of (key, count) pairs, where count is number of each key in rdd. -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list(c("a", 1), c("b", 1), c("a", 1))) -#' countByKey(rdd) # ("a", 2L), ("b", 1L) -#'} -#' @rdname countByKey -#' @aliases countByKey,RDD-method +# Count the number of elements for each key, and return the result to the +# master as lists of (key, count) pairs. +# +# Same as countByKey in Spark. +# +# @param x The RDD to count keys. +# @return list of (key, count) pairs, where count is number of each key in rdd. +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, list(c("a", 1), c("b", 1), c("a", 1))) +# countByKey(rdd) # ("a", 2L), ("b", 1L) +#} +# @rdname countByKey +# @aliases countByKey,RDD-method setMethod("countByKey", signature(x = "RDD"), function(x) { @@ -71,17 +71,17 @@ setMethod("countByKey", countByValue(keys) }) -#' Return an RDD with the keys of each tuple. -#' -#' @param x The RDD from which the keys of each tuple is returned. -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list(list(1, 2), list(3, 4))) -#' collect(keys(rdd)) # list(1, 3) -#'} -#' @rdname keys -#' @aliases keys,RDD +# Return an RDD with the keys of each tuple. +# +# @param x The RDD from which the keys of each tuple is returned. +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, list(list(1, 2), list(3, 4))) +# collect(keys(rdd)) # list(1, 3) +#} +# @rdname keys +# @aliases keys,RDD setMethod("keys", signature(x = "RDD"), function(x) { @@ -91,17 +91,17 @@ setMethod("keys", lapply(x, func) }) -#' Return an RDD with the values of each tuple. -#' -#' @param x The RDD from which the values of each tuple is returned. -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list(list(1, 2), list(3, 4))) -#' collect(values(rdd)) # list(2, 4) -#'} -#' @rdname values -#' @aliases values,RDD +# Return an RDD with the values of each tuple. +# +# @param x The RDD from which the values of each tuple is returned. +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, list(list(1, 2), list(3, 4))) +# collect(values(rdd)) # list(2, 4) +#} +# @rdname values +# @aliases values,RDD setMethod("values", signature(x = "RDD"), function(x) { @@ -111,23 +111,23 @@ setMethod("values", lapply(x, func) }) -#' Applies a function to all values of the elements, without modifying the keys. -#' -#' The same as `mapValues()' in Spark. -#' -#' @param X The RDD to apply the transformation. -#' @param FUN the transformation to apply on the value of each element. -#' @return a new RDD created by the transformation. -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:10) -#' makePairs <- lapply(rdd, function(x) { list(x, x) }) -#' collect(mapValues(makePairs, function(x) { x * 2) }) -#' Output: list(list(1,2), list(2,4), list(3,6), ...) -#'} -#' @rdname mapValues -#' @aliases mapValues,RDD,function-method +# Applies a function to all values of the elements, without modifying the keys. +# +# The same as `mapValues()' in Spark. +# +# @param X The RDD to apply the transformation. +# @param FUN the transformation to apply on the value of each element. +# @return a new RDD created by the transformation. +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, 1:10) +# makePairs <- lapply(rdd, function(x) { list(x, x) }) +# collect(mapValues(makePairs, function(x) { x * 2) }) +# Output: list(list(1,2), list(2,4), list(3,6), ...) +#} +# @rdname mapValues +# @aliases mapValues,RDD,function-method setMethod("mapValues", signature(X = "RDD", FUN = "function"), function(X, FUN) { @@ -137,23 +137,23 @@ setMethod("mapValues", lapply(X, func) }) -#' Pass each value in the key-value pair RDD through a flatMap function without -#' changing the keys; this also retains the original RDD's partitioning. -#' -#' The same as 'flatMapValues()' in Spark. -#' -#' @param X The RDD to apply the transformation. -#' @param FUN the transformation to apply on the value of each element. -#' @return a new RDD created by the transformation. -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list(list(1, c(1,2)), list(2, c(3,4)))) -#' collect(flatMapValues(rdd, function(x) { x })) -#' Output: list(list(1,1), list(1,2), list(2,3), list(2,4)) -#'} -#' @rdname flatMapValues -#' @aliases flatMapValues,RDD,function-method +# Pass each value in the key-value pair RDD through a flatMap function without +# changing the keys; this also retains the original RDD's partitioning. +# +# The same as 'flatMapValues()' in Spark. +# +# @param X The RDD to apply the transformation. +# @param FUN the transformation to apply on the value of each element. +# @return a new RDD created by the transformation. +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, list(list(1, c(1,2)), list(2, c(3,4)))) +# collect(flatMapValues(rdd, function(x) { x })) +# Output: list(list(1,1), list(1,2), list(2,3), list(2,4)) +#} +# @rdname flatMapValues +# @aliases flatMapValues,RDD,function-method setMethod("flatMapValues", signature(X = "RDD", FUN = "function"), function(X, FUN) { @@ -165,30 +165,30 @@ setMethod("flatMapValues", ############ Shuffle Functions ############ -#' Partition an RDD by key -#' -#' This function operates on RDDs where every element is of the form list(K, V) or c(K, V). -#' For each element of this RDD, the partitioner is used to compute a hash -#' function and the RDD is partitioned using this hash value. -#' -#' @param x The RDD to partition. Should be an RDD where each element is -#' list(K, V) or c(K, V). -#' @param numPartitions Number of partitions to create. -#' @param ... Other optional arguments to partitionBy. -#' -#' @param partitionFunc The partition function to use. Uses a default hashCode -#' function if not provided -#' @return An RDD partitioned using the specified partitioner. -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' pairs <- list(list(1, 2), list(1.1, 3), list(1, 4)) -#' rdd <- parallelize(sc, pairs) -#' parts <- partitionBy(rdd, 2L) -#' collectPartition(parts, 0L) # First partition should contain list(1, 2) and list(1, 4) -#'} -#' @rdname partitionBy -#' @aliases partitionBy,RDD,integer-method +# Partition an RDD by key +# +# This function operates on RDDs where every element is of the form list(K, V) or c(K, V). +# For each element of this RDD, the partitioner is used to compute a hash +# function and the RDD is partitioned using this hash value. +# +# @param x The RDD to partition. Should be an RDD where each element is +# list(K, V) or c(K, V). +# @param numPartitions Number of partitions to create. +# @param ... Other optional arguments to partitionBy. +# +# @param partitionFunc The partition function to use. Uses a default hashCode +# function if not provided +# @return An RDD partitioned using the specified partitioner. +# @examples +#\dontrun{ +# sc <- sparkR.init() +# pairs <- list(list(1, 2), list(1.1, 3), list(1, 4)) +# rdd <- parallelize(sc, pairs) +# parts <- partitionBy(rdd, 2L) +# collectPartition(parts, 0L) # First partition should contain list(1, 2) and list(1, 4) +#} +# @rdname partitionBy +# @aliases partitionBy,RDD,integer-method setMethod("partitionBy", signature(x = "RDD", numPartitions = "numeric"), function(x, numPartitions, partitionFunc = hashCode) { @@ -234,27 +234,27 @@ setMethod("partitionBy", RDD(r, serializedMode = "byte") }) -#' Group values by key -#' -#' This function operates on RDDs where every element is of the form list(K, V) or c(K, V). -#' and group values for each key in the RDD into a single sequence. -#' -#' @param x The RDD to group. Should be an RDD where each element is -#' list(K, V) or c(K, V). -#' @param numPartitions Number of partitions to create. -#' @return An RDD where each element is list(K, list(V)) -#' @seealso reduceByKey -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' pairs <- list(list(1, 2), list(1.1, 3), list(1, 4)) -#' rdd <- parallelize(sc, pairs) -#' parts <- groupByKey(rdd, 2L) -#' grouped <- collect(parts) -#' grouped[[1]] # Should be a list(1, list(2, 4)) -#'} -#' @rdname groupByKey -#' @aliases groupByKey,RDD,integer-method +# Group values by key +# +# This function operates on RDDs where every element is of the form list(K, V) or c(K, V). +# and group values for each key in the RDD into a single sequence. +# +# @param x The RDD to group. Should be an RDD where each element is +# list(K, V) or c(K, V). +# @param numPartitions Number of partitions to create. +# @return An RDD where each element is list(K, list(V)) +# @seealso reduceByKey +# @examples +#\dontrun{ +# sc <- sparkR.init() +# pairs <- list(list(1, 2), list(1.1, 3), list(1, 4)) +# rdd <- parallelize(sc, pairs) +# parts <- groupByKey(rdd, 2L) +# grouped <- collect(parts) +# grouped[[1]] # Should be a list(1, list(2, 4)) +#} +# @rdname groupByKey +# @aliases groupByKey,RDD,integer-method setMethod("groupByKey", signature(x = "RDD", numPartitions = "numeric"), function(x, numPartitions) { @@ -292,28 +292,28 @@ setMethod("groupByKey", lapplyPartition(shuffled, groupVals) }) -#' Merge values by key -#' -#' This function operates on RDDs where every element is of the form list(K, V) or c(K, V). -#' and merges the values for each key using an associative reduce function. -#' -#' @param x The RDD to reduce by key. Should be an RDD where each element is -#' list(K, V) or c(K, V). -#' @param combineFunc The associative reduce function to use. -#' @param numPartitions Number of partitions to create. -#' @return An RDD where each element is list(K, V') where V' is the merged -#' value -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' pairs <- list(list(1, 2), list(1.1, 3), list(1, 4)) -#' rdd <- parallelize(sc, pairs) -#' parts <- reduceByKey(rdd, "+", 2L) -#' reduced <- collect(parts) -#' reduced[[1]] # Should be a list(1, 6) -#'} -#' @rdname reduceByKey -#' @aliases reduceByKey,RDD,integer-method +# Merge values by key +# +# This function operates on RDDs where every element is of the form list(K, V) or c(K, V). +# and merges the values for each key using an associative reduce function. +# +# @param x The RDD to reduce by key. Should be an RDD where each element is +# list(K, V) or c(K, V). +# @param combineFunc The associative reduce function to use. +# @param numPartitions Number of partitions to create. +# @return An RDD where each element is list(K, V') where V' is the merged +# value +# @examples +#\dontrun{ +# sc <- sparkR.init() +# pairs <- list(list(1, 2), list(1.1, 3), list(1, 4)) +# rdd <- parallelize(sc, pairs) +# parts <- reduceByKey(rdd, "+", 2L) +# reduced <- collect(parts) +# reduced[[1]] # Should be a list(1, 6) +#} +# @rdname reduceByKey +# @aliases reduceByKey,RDD,integer-method setMethod("reduceByKey", signature(x = "RDD", combineFunc = "ANY", numPartitions = "numeric"), function(x, combineFunc, numPartitions) { @@ -333,27 +333,27 @@ setMethod("reduceByKey", lapplyPartition(shuffled, reduceVals) }) -#' Merge values by key locally -#' -#' This function operates on RDDs where every element is of the form list(K, V) or c(K, V). -#' and merges the values for each key using an associative reduce function, but return the -#' results immediately to the driver as an R list. -#' -#' @param x The RDD to reduce by key. Should be an RDD where each element is -#' list(K, V) or c(K, V). -#' @param combineFunc The associative reduce function to use. -#' @return A list of elements of type list(K, V') where V' is the merged value for each key -#' @seealso reduceByKey -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' pairs <- list(list(1, 2), list(1.1, 3), list(1, 4)) -#' rdd <- parallelize(sc, pairs) -#' reduced <- reduceByKeyLocally(rdd, "+") -#' reduced # list(list(1, 6), list(1.1, 3)) -#'} -#' @rdname reduceByKeyLocally -#' @aliases reduceByKeyLocally,RDD,integer-method +# Merge values by key locally +# +# This function operates on RDDs where every element is of the form list(K, V) or c(K, V). +# and merges the values for each key using an associative reduce function, but return the +# results immediately to the driver as an R list. +# +# @param x The RDD to reduce by key. Should be an RDD where each element is +# list(K, V) or c(K, V). +# @param combineFunc The associative reduce function to use. +# @return A list of elements of type list(K, V') where V' is the merged value for each key +# @seealso reduceByKey +# @examples +#\dontrun{ +# sc <- sparkR.init() +# pairs <- list(list(1, 2), list(1.1, 3), list(1, 4)) +# rdd <- parallelize(sc, pairs) +# reduced <- reduceByKeyLocally(rdd, "+") +# reduced # list(list(1, 6), list(1.1, 3)) +#} +# @rdname reduceByKeyLocally +# @aliases reduceByKeyLocally,RDD,integer-method setMethod("reduceByKeyLocally", signature(x = "RDD", combineFunc = "ANY"), function(x, combineFunc) { @@ -385,41 +385,41 @@ setMethod("reduceByKeyLocally", convertEnvsToList(merged[[1]], merged[[2]]) }) -#' Combine values by key -#' -#' Generic function to combine the elements for each key using a custom set of -#' aggregation functions. Turns an RDD[(K, V)] into a result of type RDD[(K, C)], -#' for a "combined type" C. Note that V and C can be different -- for example, one -#' might group an RDD of type (Int, Int) into an RDD of type (Int, Seq[Int]). - -#' Users provide three functions: -#' \itemize{ -#' \item createCombiner, which turns a V into a C (e.g., creates a one-element list) -#' \item mergeValue, to merge a V into a C (e.g., adds it to the end of a list) - -#' \item mergeCombiners, to combine two C's into a single one (e.g., concatentates -#' two lists). -#' } -#' -#' @param x The RDD to combine. Should be an RDD where each element is -#' list(K, V) or c(K, V). -#' @param createCombiner Create a combiner (C) given a value (V) -#' @param mergeValue Merge the given value (V) with an existing combiner (C) -#' @param mergeCombiners Merge two combiners and return a new combiner -#' @param numPartitions Number of partitions to create. -#' @return An RDD where each element is list(K, C) where C is the combined type -#' -#' @seealso groupByKey, reduceByKey -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' pairs <- list(list(1, 2), list(1.1, 3), list(1, 4)) -#' rdd <- parallelize(sc, pairs) -#' parts <- combineByKey(rdd, function(x) { x }, "+", "+", 2L) -#' combined <- collect(parts) -#' combined[[1]] # Should be a list(1, 6) -#'} -#' @rdname combineByKey -#' @aliases combineByKey,RDD,ANY,ANY,ANY,integer-method +# Combine values by key +# +# Generic function to combine the elements for each key using a custom set of +# aggregation functions. Turns an RDD[(K, V)] into a result of type RDD[(K, C)], +# for a "combined type" C. Note that V and C can be different -- for example, one +# might group an RDD of type (Int, Int) into an RDD of type (Int, Seq[Int]). + +# Users provide three functions: +# \itemize{ +# \item createCombiner, which turns a V into a C (e.g., creates a one-element list) +# \item mergeValue, to merge a V into a C (e.g., adds it to the end of a list) - +# \item mergeCombiners, to combine two C's into a single one (e.g., concatentates +# two lists). +# } +# +# @param x The RDD to combine. Should be an RDD where each element is +# list(K, V) or c(K, V). +# @param createCombiner Create a combiner (C) given a value (V) +# @param mergeValue Merge the given value (V) with an existing combiner (C) +# @param mergeCombiners Merge two combiners and return a new combiner +# @param numPartitions Number of partitions to create. +# @return An RDD where each element is list(K, C) where C is the combined type +# +# @seealso groupByKey, reduceByKey +# @examples +#\dontrun{ +# sc <- sparkR.init() +# pairs <- list(list(1, 2), list(1.1, 3), list(1, 4)) +# rdd <- parallelize(sc, pairs) +# parts <- combineByKey(rdd, function(x) { x }, "+", "+", 2L) +# combined <- collect(parts) +# combined[[1]] # Should be a list(1, 6) +#} +# @rdname combineByKey +# @aliases combineByKey,RDD,ANY,ANY,ANY,integer-method setMethod("combineByKey", signature(x = "RDD", createCombiner = "ANY", mergeValue = "ANY", mergeCombiners = "ANY", numPartitions = "numeric"), @@ -451,36 +451,36 @@ setMethod("combineByKey", lapplyPartition(shuffled, mergeAfterShuffle) }) -#' Aggregate a pair RDD by each key. -#' -#' Aggregate the values of each key in an RDD, using given combine functions -#' and a neutral "zero value". This function can return a different result type, -#' U, than the type of the values in this RDD, V. Thus, we need one operation -#' for merging a V into a U and one operation for merging two U's, The former -#' operation is used for merging values within a partition, and the latter is -#' used for merging values between partitions. To avoid memory allocation, both -#' of these functions are allowed to modify and return their first argument -#' instead of creating a new U. -#' -#' @param x An RDD. -#' @param zeroValue A neutral "zero value". -#' @param seqOp A function to aggregate the values of each key. It may return -#' a different result type from the type of the values. -#' @param combOp A function to aggregate results of seqOp. -#' @return An RDD containing the aggregation result. -#' @seealso foldByKey, combineByKey -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list(list(1, 1), list(1, 2), list(2, 3), list(2, 4))) -#' zeroValue <- list(0, 0) -#' seqOp <- function(x, y) { list(x[[1]] + y, x[[2]] + 1) } -#' combOp <- function(x, y) { list(x[[1]] + y[[1]], x[[2]] + y[[2]]) } -#' aggregateByKey(rdd, zeroValue, seqOp, combOp, 2L) -#' # list(list(1, list(3, 2)), list(2, list(7, 2))) -#'} -#' @rdname aggregateByKey -#' @aliases aggregateByKey,RDD,ANY,ANY,ANY,integer-method +# Aggregate a pair RDD by each key. +# +# Aggregate the values of each key in an RDD, using given combine functions +# and a neutral "zero value". This function can return a different result type, +# U, than the type of the values in this RDD, V. Thus, we need one operation +# for merging a V into a U and one operation for merging two U's, The former +# operation is used for merging values within a partition, and the latter is +# used for merging values between partitions. To avoid memory allocation, both +# of these functions are allowed to modify and return their first argument +# instead of creating a new U. +# +# @param x An RDD. +# @param zeroValue A neutral "zero value". +# @param seqOp A function to aggregate the values of each key. It may return +# a different result type from the type of the values. +# @param combOp A function to aggregate results of seqOp. +# @return An RDD containing the aggregation result. +# @seealso foldByKey, combineByKey +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, list(list(1, 1), list(1, 2), list(2, 3), list(2, 4))) +# zeroValue <- list(0, 0) +# seqOp <- function(x, y) { list(x[[1]] + y, x[[2]] + 1) } +# combOp <- function(x, y) { list(x[[1]] + y[[1]], x[[2]] + y[[2]]) } +# aggregateByKey(rdd, zeroValue, seqOp, combOp, 2L) +# # list(list(1, list(3, 2)), list(2, list(7, 2))) +#} +# @rdname aggregateByKey +# @aliases aggregateByKey,RDD,ANY,ANY,ANY,integer-method setMethod("aggregateByKey", signature(x = "RDD", zeroValue = "ANY", seqOp = "ANY", combOp = "ANY", numPartitions = "numeric"), @@ -492,26 +492,26 @@ setMethod("aggregateByKey", combineByKey(x, createCombiner, seqOp, combOp, numPartitions) }) -#' Fold a pair RDD by each key. -#' -#' Aggregate the values of each key in an RDD, using an associative function "func" -#' and a neutral "zero value" which may be added to the result an arbitrary -#' number of times, and must not change the result (e.g., 0 for addition, or -#' 1 for multiplication.). -#' -#' @param x An RDD. -#' @param zeroValue A neutral "zero value". -#' @param func An associative function for folding values of each key. -#' @return An RDD containing the aggregation result. -#' @seealso aggregateByKey, combineByKey -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list(list(1, 1), list(1, 2), list(2, 3), list(2, 4))) -#' foldByKey(rdd, 0, "+", 2L) # list(list(1, 3), list(2, 7)) -#'} -#' @rdname foldByKey -#' @aliases foldByKey,RDD,ANY,ANY,integer-method +# Fold a pair RDD by each key. +# +# Aggregate the values of each key in an RDD, using an associative function "func" +# and a neutral "zero value" which may be added to the result an arbitrary +# number of times, and must not change the result (e.g., 0 for addition, or +# 1 for multiplication.). +# +# @param x An RDD. +# @param zeroValue A neutral "zero value". +# @param func An associative function for folding values of each key. +# @return An RDD containing the aggregation result. +# @seealso aggregateByKey, combineByKey +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, list(list(1, 1), list(1, 2), list(2, 3), list(2, 4))) +# foldByKey(rdd, 0, "+", 2L) # list(list(1, 3), list(2, 7)) +#} +# @rdname foldByKey +# @aliases foldByKey,RDD,ANY,ANY,integer-method setMethod("foldByKey", signature(x = "RDD", zeroValue = "ANY", func = "ANY", numPartitions = "numeric"), @@ -521,28 +521,28 @@ setMethod("foldByKey", ############ Binary Functions ############# -#' Join two RDDs -#' -#' @description -#' \code{join} This function joins two RDDs where every element is of the form list(K, V). -#' The key types of the two RDDs should be the same. -#' -#' @param x An RDD to be joined. Should be an RDD where each element is -#' list(K, V). -#' @param y An RDD to be joined. Should be an RDD where each element is -#' list(K, V). -#' @param numPartitions Number of partitions to create. -#' @return a new RDD containing all pairs of elements with matching keys in -#' two input RDDs. -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd1 <- parallelize(sc, list(list(1, 1), list(2, 4))) -#' rdd2 <- parallelize(sc, list(list(1, 2), list(1, 3))) -#' join(rdd1, rdd2, 2L) # list(list(1, list(1, 2)), list(1, list(1, 3)) -#'} -#' @rdname join-methods -#' @aliases join,RDD,RDD-method +# Join two RDDs +# +# @description +# \code{join} This function joins two RDDs where every element is of the form list(K, V). +# The key types of the two RDDs should be the same. +# +# @param x An RDD to be joined. Should be an RDD where each element is +# list(K, V). +# @param y An RDD to be joined. Should be an RDD where each element is +# list(K, V). +# @param numPartitions Number of partitions to create. +# @return a new RDD containing all pairs of elements with matching keys in +# two input RDDs. +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd1 <- parallelize(sc, list(list(1, 1), list(2, 4))) +# rdd2 <- parallelize(sc, list(list(1, 2), list(1, 3))) +# join(rdd1, rdd2, 2L) # list(list(1, list(1, 2)), list(1, list(1, 3)) +#} +# @rdname join-methods +# @aliases join,RDD,RDD-method setMethod("join", signature(x = "RDD", y = "RDD"), function(x, y, numPartitions) { @@ -557,30 +557,30 @@ setMethod("join", doJoin) }) -#' Left outer join two RDDs -#' -#' @description -#' \code{leftouterjoin} This function left-outer-joins two RDDs where every element is of the form list(K, V). -#' The key types of the two RDDs should be the same. -#' -#' @param x An RDD to be joined. Should be an RDD where each element is -#' list(K, V). -#' @param y An RDD to be joined. Should be an RDD where each element is -#' list(K, V). -#' @param numPartitions Number of partitions to create. -#' @return For each element (k, v) in x, the resulting RDD will either contain -#' all pairs (k, (v, w)) for (k, w) in rdd2, or the pair (k, (v, NULL)) -#' if no elements in rdd2 have key k. -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd1 <- parallelize(sc, list(list(1, 1), list(2, 4))) -#' rdd2 <- parallelize(sc, list(list(1, 2), list(1, 3))) -#' leftOuterJoin(rdd1, rdd2, 2L) -#' # list(list(1, list(1, 2)), list(1, list(1, 3)), list(2, list(4, NULL))) -#'} -#' @rdname join-methods -#' @aliases leftOuterJoin,RDD,RDD-method +# Left outer join two RDDs +# +# @description +# \code{leftouterjoin} This function left-outer-joins two RDDs where every element is of the form list(K, V). +# The key types of the two RDDs should be the same. +# +# @param x An RDD to be joined. Should be an RDD where each element is +# list(K, V). +# @param y An RDD to be joined. Should be an RDD where each element is +# list(K, V). +# @param numPartitions Number of partitions to create. +# @return For each element (k, v) in x, the resulting RDD will either contain +# all pairs (k, (v, w)) for (k, w) in rdd2, or the pair (k, (v, NULL)) +# if no elements in rdd2 have key k. +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd1 <- parallelize(sc, list(list(1, 1), list(2, 4))) +# rdd2 <- parallelize(sc, list(list(1, 2), list(1, 3))) +# leftOuterJoin(rdd1, rdd2, 2L) +# # list(list(1, list(1, 2)), list(1, list(1, 3)), list(2, list(4, NULL))) +#} +# @rdname join-methods +# @aliases leftOuterJoin,RDD,RDD-method setMethod("leftOuterJoin", signature(x = "RDD", y = "RDD", numPartitions = "numeric"), function(x, y, numPartitions) { @@ -594,30 +594,30 @@ setMethod("leftOuterJoin", joined <- flatMapValues(groupByKey(unionRDD(xTagged, yTagged), numPartitions), doJoin) }) -#' Right outer join two RDDs -#' -#' @description -#' \code{rightouterjoin} This function right-outer-joins two RDDs where every element is of the form list(K, V). -#' The key types of the two RDDs should be the same. -#' -#' @param x An RDD to be joined. Should be an RDD where each element is -#' list(K, V). -#' @param y An RDD to be joined. Should be an RDD where each element is -#' list(K, V). -#' @param numPartitions Number of partitions to create. -#' @return For each element (k, w) in y, the resulting RDD will either contain -#' all pairs (k, (v, w)) for (k, v) in x, or the pair (k, (NULL, w)) -#' if no elements in x have key k. -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd1 <- parallelize(sc, list(list(1, 2), list(1, 3))) -#' rdd2 <- parallelize(sc, list(list(1, 1), list(2, 4))) -#' rightOuterJoin(rdd1, rdd2, 2L) -#' # list(list(1, list(2, 1)), list(1, list(3, 1)), list(2, list(NULL, 4))) -#'} -#' @rdname join-methods -#' @aliases rightOuterJoin,RDD,RDD-method +# Right outer join two RDDs +# +# @description +# \code{rightouterjoin} This function right-outer-joins two RDDs where every element is of the form list(K, V). +# The key types of the two RDDs should be the same. +# +# @param x An RDD to be joined. Should be an RDD where each element is +# list(K, V). +# @param y An RDD to be joined. Should be an RDD where each element is +# list(K, V). +# @param numPartitions Number of partitions to create. +# @return For each element (k, w) in y, the resulting RDD will either contain +# all pairs (k, (v, w)) for (k, v) in x, or the pair (k, (NULL, w)) +# if no elements in x have key k. +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd1 <- parallelize(sc, list(list(1, 2), list(1, 3))) +# rdd2 <- parallelize(sc, list(list(1, 1), list(2, 4))) +# rightOuterJoin(rdd1, rdd2, 2L) +# # list(list(1, list(2, 1)), list(1, list(3, 1)), list(2, list(NULL, 4))) +#} +# @rdname join-methods +# @aliases rightOuterJoin,RDD,RDD-method setMethod("rightOuterJoin", signature(x = "RDD", y = "RDD", numPartitions = "numeric"), function(x, y, numPartitions) { @@ -631,33 +631,33 @@ setMethod("rightOuterJoin", joined <- flatMapValues(groupByKey(unionRDD(xTagged, yTagged), numPartitions), doJoin) }) -#' Full outer join two RDDs -#' -#' @description -#' \code{fullouterjoin} This function full-outer-joins two RDDs where every element is of the form list(K, V). -#' The key types of the two RDDs should be the same. -#' -#' @param x An RDD to be joined. Should be an RDD where each element is -#' list(K, V). -#' @param y An RDD to be joined. Should be an RDD where each element is -#' list(K, V). -#' @param numPartitions Number of partitions to create. -#' @return For each element (k, v) in x and (k, w) in y, the resulting RDD -#' will contain all pairs (k, (v, w)) for both (k, v) in x and -#' (k, w) in y, or the pair (k, (NULL, w))/(k, (v, NULL)) if no elements -#' in x/y have key k. -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd1 <- parallelize(sc, list(list(1, 2), list(1, 3), list(3, 3))) -#' rdd2 <- parallelize(sc, list(list(1, 1), list(2, 4))) -#' fullOuterJoin(rdd1, rdd2, 2L) # list(list(1, list(2, 1)), -#' # list(1, list(3, 1)), -#' # list(2, list(NULL, 4))) -#' # list(3, list(3, NULL)), -#'} -#' @rdname join-methods -#' @aliases fullOuterJoin,RDD,RDD-method +# Full outer join two RDDs +# +# @description +# \code{fullouterjoin} This function full-outer-joins two RDDs where every element is of the form list(K, V). +# The key types of the two RDDs should be the same. +# +# @param x An RDD to be joined. Should be an RDD where each element is +# list(K, V). +# @param y An RDD to be joined. Should be an RDD where each element is +# list(K, V). +# @param numPartitions Number of partitions to create. +# @return For each element (k, v) in x and (k, w) in y, the resulting RDD +# will contain all pairs (k, (v, w)) for both (k, v) in x and +# (k, w) in y, or the pair (k, (NULL, w))/(k, (v, NULL)) if no elements +# in x/y have key k. +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd1 <- parallelize(sc, list(list(1, 2), list(1, 3), list(3, 3))) +# rdd2 <- parallelize(sc, list(list(1, 1), list(2, 4))) +# fullOuterJoin(rdd1, rdd2, 2L) # list(list(1, list(2, 1)), +# # list(1, list(3, 1)), +# # list(2, list(NULL, 4))) +# # list(3, list(3, NULL)), +#} +# @rdname join-methods +# @aliases fullOuterJoin,RDD,RDD-method setMethod("fullOuterJoin", signature(x = "RDD", y = "RDD", numPartitions = "numeric"), function(x, y, numPartitions) { @@ -671,23 +671,23 @@ setMethod("fullOuterJoin", joined <- flatMapValues(groupByKey(unionRDD(xTagged, yTagged), numPartitions), doJoin) }) -#' For each key k in several RDDs, return a resulting RDD that -#' whose values are a list of values for the key in all RDDs. -#' -#' @param ... Several RDDs. -#' @param numPartitions Number of partitions to create. -#' @return a new RDD containing all pairs of elements with values in a list -#' in all RDDs. -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd1 <- parallelize(sc, list(list(1, 1), list(2, 4))) -#' rdd2 <- parallelize(sc, list(list(1, 2), list(1, 3))) -#' cogroup(rdd1, rdd2, numPartitions = 2L) -#' # list(list(1, list(1, list(2, 3))), list(2, list(list(4), list())) -#'} -#' @rdname cogroup -#' @aliases cogroup,RDD-method +# For each key k in several RDDs, return a resulting RDD that +# whose values are a list of values for the key in all RDDs. +# +# @param ... Several RDDs. +# @param numPartitions Number of partitions to create. +# @return a new RDD containing all pairs of elements with values in a list +# in all RDDs. +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd1 <- parallelize(sc, list(list(1, 1), list(2, 4))) +# rdd2 <- parallelize(sc, list(list(1, 2), list(1, 3))) +# cogroup(rdd1, rdd2, numPartitions = 2L) +# # list(list(1, list(1, list(2, 3))), list(2, list(list(4), list())) +#} +# @rdname cogroup +# @aliases cogroup,RDD-method setMethod("cogroup", "RDD", function(..., numPartitions) { @@ -723,20 +723,20 @@ setMethod("cogroup", group.func) }) -#' Sort a (k, v) pair RDD by k. -#' -#' @param x A (k, v) pair RDD to be sorted. -#' @param ascending A flag to indicate whether the sorting is ascending or descending. -#' @param numPartitions Number of partitions to create. -#' @return An RDD where all (k, v) pair elements are sorted. -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, list(list(3, 1), list(2, 2), list(1, 3))) -#' collect(sortByKey(rdd)) # list (list(1, 3), list(2, 2), list(3, 1)) -#'} -#' @rdname sortByKey -#' @aliases sortByKey,RDD,RDD-method +# Sort a (k, v) pair RDD by k. +# +# @param x A (k, v) pair RDD to be sorted. +# @param ascending A flag to indicate whether the sorting is ascending or descending. +# @param numPartitions Number of partitions to create. +# @return An RDD where all (k, v) pair elements are sorted. +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, list(list(3, 1), list(2, 2), list(1, 3))) +# collect(sortByKey(rdd)) # list (list(1, 3), list(2, 2), list(3, 1)) +#} +# @rdname sortByKey +# @aliases sortByKey,RDD,RDD-method setMethod("sortByKey", signature(x = "RDD"), function(x, ascending = TRUE, numPartitions = SparkR:::numPartitions(x)) { @@ -785,25 +785,25 @@ setMethod("sortByKey", lapplyPartition(newRDD, partitionFunc) }) -#' Subtract a pair RDD with another pair RDD. -#' -#' Return an RDD with the pairs from x whose keys are not in other. -#' -#' @param x An RDD. -#' @param other An RDD. -#' @param numPartitions Number of the partitions in the result RDD. -#' @return An RDD with the pairs from x whose keys are not in other. -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd1 <- parallelize(sc, list(list("a", 1), list("b", 4), -#' list("b", 5), list("a", 2))) -#' rdd2 <- parallelize(sc, list(list("a", 3), list("c", 1))) -#' collect(subtractByKey(rdd1, rdd2)) -#' # list(list("b", 4), list("b", 5)) -#'} -#' @rdname subtractByKey -#' @aliases subtractByKey,RDD +# Subtract a pair RDD with another pair RDD. +# +# Return an RDD with the pairs from x whose keys are not in other. +# +# @param x An RDD. +# @param other An RDD. +# @param numPartitions Number of the partitions in the result RDD. +# @return An RDD with the pairs from x whose keys are not in other. +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd1 <- parallelize(sc, list(list("a", 1), list("b", 4), +# list("b", 5), list("a", 2))) +# rdd2 <- parallelize(sc, list(list("a", 3), list("c", 1))) +# collect(subtractByKey(rdd1, rdd2)) +# # list(list("b", 4), list("b", 5)) +#} +# @rdname subtractByKey +# @aliases subtractByKey,RDD setMethod("subtractByKey", signature(x = "RDD", other = "RDD"), function(x, other, numPartitions = SparkR:::numPartitions(x)) { @@ -819,41 +819,41 @@ setMethod("subtractByKey", function (v) { v[[1]] }) }) -#' Return a subset of this RDD sampled by key. -#' -#' @description -#' \code{sampleByKey} Create a sample of this RDD using variable sampling rates -#' for different keys as specified by fractions, a key to sampling rate map. -#' -#' @param x The RDD to sample elements by key, where each element is -#' list(K, V) or c(K, V). -#' @param withReplacement Sampling with replacement or not -#' @param fraction The (rough) sample target fraction -#' @param seed Randomness seed value -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' rdd <- parallelize(sc, 1:3000) -#' pairs <- lapply(rdd, function(x) { if (x %% 3 == 0) list("a", x) -#' else { if (x %% 3 == 1) list("b", x) else list("c", x) }}) -#' fractions <- list(a = 0.2, b = 0.1, c = 0.3) -#' sample <- sampleByKey(pairs, FALSE, fractions, 1618L) -#' 100 < length(lookup(sample, "a")) && 300 > length(lookup(sample, "a")) # TRUE -#' 50 < length(lookup(sample, "b")) && 150 > length(lookup(sample, "b")) # TRUE -#' 200 < length(lookup(sample, "c")) && 400 > length(lookup(sample, "c")) # TRUE -#' lookup(sample, "a")[which.min(lookup(sample, "a"))] >= 0 # TRUE -#' lookup(sample, "a")[which.max(lookup(sample, "a"))] <= 2000 # TRUE -#' lookup(sample, "b")[which.min(lookup(sample, "b"))] >= 0 # TRUE -#' lookup(sample, "b")[which.max(lookup(sample, "b"))] <= 2000 # TRUE -#' lookup(sample, "c")[which.min(lookup(sample, "c"))] >= 0 # TRUE -#' lookup(sample, "c")[which.max(lookup(sample, "c"))] <= 2000 # TRUE -#' fractions <- list(a = 0.2, b = 0.1, c = 0.3, d = 0.4) -#' sample <- sampleByKey(pairs, FALSE, fractions, 1618L) # Key "d" will be ignored -#' fractions <- list(a = 0.2, b = 0.1) -#' sample <- sampleByKey(pairs, FALSE, fractions, 1618L) # KeyError: "c" -#'} -#' @rdname sampleByKey -#' @aliases sampleByKey,RDD-method +# Return a subset of this RDD sampled by key. +# +# @description +# \code{sampleByKey} Create a sample of this RDD using variable sampling rates +# for different keys as specified by fractions, a key to sampling rate map. +# +# @param x The RDD to sample elements by key, where each element is +# list(K, V) or c(K, V). +# @param withReplacement Sampling with replacement or not +# @param fraction The (rough) sample target fraction +# @param seed Randomness seed value +# @examples +#\dontrun{ +# sc <- sparkR.init() +# rdd <- parallelize(sc, 1:3000) +# pairs <- lapply(rdd, function(x) { if (x %% 3 == 0) list("a", x) +# else { if (x %% 3 == 1) list("b", x) else list("c", x) }}) +# fractions <- list(a = 0.2, b = 0.1, c = 0.3) +# sample <- sampleByKey(pairs, FALSE, fractions, 1618L) +# 100 < length(lookup(sample, "a")) && 300 > length(lookup(sample, "a")) # TRUE +# 50 < length(lookup(sample, "b")) && 150 > length(lookup(sample, "b")) # TRUE +# 200 < length(lookup(sample, "c")) && 400 > length(lookup(sample, "c")) # TRUE +# lookup(sample, "a")[which.min(lookup(sample, "a"))] >= 0 # TRUE +# lookup(sample, "a")[which.max(lookup(sample, "a"))] <= 2000 # TRUE +# lookup(sample, "b")[which.min(lookup(sample, "b"))] >= 0 # TRUE +# lookup(sample, "b")[which.max(lookup(sample, "b"))] <= 2000 # TRUE +# lookup(sample, "c")[which.min(lookup(sample, "c"))] >= 0 # TRUE +# lookup(sample, "c")[which.max(lookup(sample, "c"))] <= 2000 # TRUE +# fractions <- list(a = 0.2, b = 0.1, c = 0.3, d = 0.4) +# sample <- sampleByKey(pairs, FALSE, fractions, 1618L) # Key "d" will be ignored +# fractions <- list(a = 0.2, b = 0.1) +# sample <- sampleByKey(pairs, FALSE, fractions, 1618L) # KeyError: "c" +#} +# @rdname sampleByKey +# @aliases sampleByKey,RDD-method setMethod("sampleByKey", signature(x = "RDD", withReplacement = "logical", fractions = "vector", seed = "integer"), From 35d6a99cbe3f67da5d56888e63baf9bc69f3de91 Mon Sep 17 00:00:00 2001 From: Jacek Lewandowski Date: Fri, 8 May 2015 11:36:30 -0700 Subject: [PATCH 63/75] [SPARK-7436] Fixed instantiation of custom recovery mode factory and added tests Author: Jacek Lewandowski Closes #5977 from jacek-lewandowski/SPARK-7436 and squashes the following commits: ff0a3c2 [Jacek Lewandowski] SPARK-7436: Fixed instantiation of custom recovery mode factory and added tests --- .../apache/spark/deploy/master/Master.scala | 2 +- .../master/CustomRecoveryModeFactory.scala | 110 ++++++++++++++++++ .../spark/deploy/master/MasterSuite.scala | 100 +++++++++++++++- 3 files changed, 208 insertions(+), 4 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/deploy/master/CustomRecoveryModeFactory.scala diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 53e1903a3d125..fccceb3ea528b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -165,7 +165,7 @@ private[master] class Master( (fsFactory.createPersistenceEngine(), fsFactory.createLeaderElectionAgent(this)) case "CUSTOM" => val clazz = Class.forName(conf.get("spark.deploy.recoveryMode.factory")) - val factory = clazz.getConstructor(conf.getClass, Serialization.getClass) + val factory = clazz.getConstructor(classOf[SparkConf], classOf[Serialization]) .newInstance(conf, SerializationExtension(context.system)) .asInstanceOf[StandaloneRecoveryModeFactory] (factory.createPersistenceEngine(), factory.createLeaderElectionAgent(this)) diff --git a/core/src/test/scala/org/apache/spark/deploy/master/CustomRecoveryModeFactory.scala b/core/src/test/scala/org/apache/spark/deploy/master/CustomRecoveryModeFactory.scala new file mode 100644 index 0000000000000..f4e56632e426a --- /dev/null +++ b/core/src/test/scala/org/apache/spark/deploy/master/CustomRecoveryModeFactory.scala @@ -0,0 +1,110 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// This file is placed in different package to make sure all of these components work well +// when they are outside of org.apache.spark. +package other.supplier + +import scala.collection.mutable +import scala.reflect.ClassTag + +import akka.serialization.Serialization + +import org.apache.spark.SparkConf +import org.apache.spark.deploy.master._ + +class CustomRecoveryModeFactory( + conf: SparkConf, + serialization: Serialization +) extends StandaloneRecoveryModeFactory(conf, serialization) { + + CustomRecoveryModeFactory.instantiationAttempts += 1 + + /** + * PersistenceEngine defines how the persistent data(Information about worker, driver etc..) + * is handled for recovery. + * + */ + override def createPersistenceEngine(): PersistenceEngine = + new CustomPersistenceEngine(serialization) + + /** + * Create an instance of LeaderAgent that decides who gets elected as master. + */ + override def createLeaderElectionAgent(master: LeaderElectable): LeaderElectionAgent = + new CustomLeaderElectionAgent(master) +} + +object CustomRecoveryModeFactory { + @volatile var instantiationAttempts = 0 +} + +class CustomPersistenceEngine(serialization: Serialization) extends PersistenceEngine { + val data = mutable.HashMap[String, Array[Byte]]() + + CustomPersistenceEngine.lastInstance = Some(this) + + /** + * Defines how the object is serialized and persisted. Implementation will + * depend on the store used. + */ + override def persist(name: String, obj: Object): Unit = { + CustomPersistenceEngine.persistAttempts += 1 + serialization.serialize(obj) match { + case util.Success(bytes) => data += name -> bytes + case util.Failure(cause) => throw new RuntimeException(cause) + } + } + + /** + * Defines how the object referred by its name is removed from the store. + */ + override def unpersist(name: String): Unit = { + CustomPersistenceEngine.unpersistAttempts += 1 + data -= name + } + + /** + * Gives all objects, matching a prefix. This defines how objects are + * read/deserialized back. + */ + override def read[T: ClassTag](prefix: String): Seq[T] = { + CustomPersistenceEngine.readAttempts += 1 + val clazz = implicitly[ClassTag[T]].runtimeClass.asInstanceOf[Class[T]] + val results = for ((name, bytes) <- data; if name.startsWith(prefix)) + yield serialization.deserialize(bytes, clazz) + + results.find(_.isFailure).foreach { + case util.Failure(cause) => throw new RuntimeException(cause) + } + + results.flatMap(_.toOption).toSeq + } +} + +object CustomPersistenceEngine { + @volatile var persistAttempts = 0 + @volatile var unpersistAttempts = 0 + @volatile var readAttempts = 0 + + @volatile var lastInstance: Option[CustomPersistenceEngine] = None +} + +class CustomLeaderElectionAgent(val masterActor: LeaderElectable) extends LeaderElectionAgent { + masterActor.electedLeader() +} + diff --git a/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala b/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala index 34c74d87f0a62..0faa8f650e5e1 100644 --- a/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala @@ -17,12 +17,20 @@ package org.apache.spark.deploy.master +import java.util.Date + +import scala.concurrent.Await +import scala.concurrent.duration._ +import scala.language.postfixOps + import akka.actor.Address -import org.scalatest.FunSuite +import org.scalatest.{FunSuite, Matchers} +import other.supplier.{CustomPersistenceEngine, CustomRecoveryModeFactory} -import org.apache.spark.{SSLOptions, SparkConf, SparkException} +import org.apache.spark.deploy._ +import org.apache.spark.{SparkConf, SparkException} -class MasterSuite extends FunSuite { +class MasterSuite extends FunSuite with Matchers { test("toAkkaUrl") { val conf = new SparkConf(loadDefaults = false) @@ -63,4 +71,90 @@ class MasterSuite extends FunSuite { } assert("Invalid master URL: spark://1.2. 3.4:1234" === e.getMessage) } + + test("can use a custom recovery mode factory") { + val conf = new SparkConf(loadDefaults = false) + conf.set("spark.deploy.recoveryMode", "CUSTOM") + conf.set("spark.deploy.recoveryMode.factory", + classOf[CustomRecoveryModeFactory].getCanonicalName) + + val instantiationAttempts = CustomRecoveryModeFactory.instantiationAttempts + + val commandToPersist = new Command( + mainClass = "", + arguments = Nil, + environment = Map.empty, + classPathEntries = Nil, + libraryPathEntries = Nil, + javaOpts = Nil + ) + + val appToPersist = new ApplicationInfo( + startTime = 0, + id = "test_app", + desc = new ApplicationDescription( + name = "", + maxCores = None, + memoryPerExecutorMB = 0, + command = commandToPersist, + appUiUrl = "", + eventLogDir = None, + eventLogCodec = None, + coresPerExecutor = None), + submitDate = new Date(), + driver = null, + defaultCores = 0 + ) + + val driverToPersist = new DriverInfo( + startTime = 0, + id = "test_driver", + desc = new DriverDescription( + jarUrl = "", + mem = 0, + cores = 0, + supervise = false, + command = commandToPersist + ), + submitDate = new Date() + ) + + val workerToPersist = new WorkerInfo( + id = "test_worker", + host = "127.0.0.1", + port = 10000, + cores = 0, + memory = 0, + actor = null, + webUiPort = 0, + publicAddress = "" + ) + + val (actorSystem, port, uiPort, restPort) = + Master.startSystemAndActor("127.0.0.1", 7077, 8080, conf) + + try { + Await.result(actorSystem.actorSelection("/user/Master").resolveOne(10 seconds), 10 seconds) + + CustomPersistenceEngine.lastInstance.isDefined shouldBe true + val persistenceEngine = CustomPersistenceEngine.lastInstance.get + + persistenceEngine.addApplication(appToPersist) + persistenceEngine.addDriver(driverToPersist) + persistenceEngine.addWorker(workerToPersist) + + val (apps, drivers, workers) = persistenceEngine.readPersistedData() + + apps.map(_.id) should contain(appToPersist.id) + drivers.map(_.id) should contain(driverToPersist.id) + workers.map(_.id) should contain(workerToPersist.id) + + } finally { + actorSystem.shutdown() + actorSystem.awaitTermination() + } + + CustomRecoveryModeFactory.instantiationAttempts should be > instantiationAttempts + } + } From a1ec08f7edc8d956afcfbb92d10b26b7619486e8 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Fri, 8 May 2015 14:41:42 -0400 Subject: [PATCH 64/75] [SPARK-7298] Harmonize style of new visualizations - Colors on the timeline now match the rest of the UI - The expandable buttons to show timeline view, DAG, etc are now more visible - Timeline text is smaller - DAG visualization text and colors are more consistent throughout - Fix some JavaScript style issues - Various small fixes throughout (e.g. inconsistent capitalization, some confusing names, HTML escaping, etc) Author: Matei Zaharia Closes #5942 from mateiz/ui and squashes the following commits: def38d0 [Matei Zaharia] Add some tooltips 4c5a364 [Matei Zaharia] Reduce stage and rank separation slightly 43dcbe3 [Matei Zaharia] Some updates to DAG fac734a [Matei Zaharia] tweaks 6a6705d [Matei Zaharia] More fixes 67629f5 [Matei Zaharia] Various small tweaks --- .../apache/spark/ui/static/spark-dag-viz.css | 60 +++++--- .../apache/spark/ui/static/spark-dag-viz.js | 57 ++++---- .../apache/spark/ui/static/timeline-view.css | 128 +++++++++++------- .../apache/spark/ui/static/timeline-view.js | 4 +- .../org/apache/spark/ui/static/webui.css | 36 +++-- .../scala/org/apache/spark/ui/ToolTips.scala | 19 +++ .../scala/org/apache/spark/ui/UIUtils.scala | 15 +- .../apache/spark/ui/jobs/AllJobsPage.scala | 31 +++-- .../apache/spark/ui/jobs/AllStagesPage.scala | 15 +- .../org/apache/spark/ui/jobs/JobPage.scala | 23 ++-- .../org/apache/spark/ui/jobs/StagePage.scala | 16 +-- .../spark/ui/scope/RDDOperationGraph.scala | 2 +- 12 files changed, 255 insertions(+), 151 deletions(-) diff --git a/core/src/main/resources/org/apache/spark/ui/static/spark-dag-viz.css b/core/src/main/resources/org/apache/spark/ui/static/spark-dag-viz.css index 8481710828455..18c72694f3e2d 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/spark-dag-viz.css +++ b/core/src/main/resources/org/apache/spark/ui/static/spark-dag-viz.css @@ -16,40 +16,51 @@ */ #dag-viz-graph svg path { - stroke: #444444; + stroke: #444; stroke-width: 1.5px; } #dag-viz-graph svg g.cluster rect { - stroke-width: 4px; - stroke-opacity: 0.5; + stroke-width: 1px; +} + +#dag-viz-graph svg g.node circle { + fill: #444; } -#dag-viz-graph svg g.node circle, #dag-viz-graph svg g.node rect { - fill: #444444; + fill: #C3EBFF; + stroke: #3EC0FF; + stroke-width: 1px; +} + +#dag-viz-graph svg g.node.cached circle { + fill: #444; } -#dag-viz-graph svg g.node.cached circle, #dag-viz-graph svg g.node.cached rect { - fill: #FF0000; + fill: #B3F5C5; + stroke: #56F578; + stroke-width: 1px; } /* Job page specific styles */ #dag-viz-graph svg.job marker#marker-arrow path { - fill: #444444; + fill: #333; stroke-width: 0px; } #dag-viz-graph svg.job g.cluster rect { - fill: #FFFFFF; - stroke: #AADFFF; + fill: #A0DFFF; + stroke: #3EC0FF; + stroke-width: 1px; } #dag-viz-graph svg.job g.cluster[id*="stage"] rect { - stroke: #FFDDEE; - stroke-width: 6px; + fill: #FFFFFF; + stroke: #FF99AC; + stroke-width: 1px; } #dag-viz-graph svg.job g#cross-stage-edges path { @@ -57,27 +68,36 @@ } #dag-viz-graph svg.job g.cluster text { - fill: #AAAAAA; + fill: #333; } /* Stage page specific styles */ #dag-viz-graph svg.stage g.cluster rect { - fill: #F0F8FF; - stroke: #AADFFF; + fill: #A0DFFF; + stroke: #3EC0FF; + stroke-width: 1px; } #dag-viz-graph svg.stage g.cluster[id*="stage"] rect { fill: #FFFFFF; - stroke: #FFDDEE; - stroke-width: 6px; + stroke: #FFA6B6; + stroke-width: 1px; } #dag-viz-graph svg.stage g.node g.label text tspan { - fill: #FFFFFF; + fill: #333; } #dag-viz-graph svg.stage g.cluster text { - fill: #444444; - font-weight: bold; + fill: #333; +} + +#dag-viz-graph a, #dag-viz-graph a:hover { + text-decoration: none; +} + +#dag-viz-graph .label { + font-weight: normal; + text-shadow: none; } diff --git a/core/src/main/resources/org/apache/spark/ui/static/spark-dag-viz.js b/core/src/main/resources/org/apache/spark/ui/static/spark-dag-viz.js index a0e3e914c2547..764dd2cfcd76f 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/spark-dag-viz.js +++ b/core/src/main/resources/org/apache/spark/ui/static/spark-dag-viz.js @@ -52,9 +52,9 @@ */ var VizConstants = { - svgMarginX: 20, - svgMarginY: 20, - stageSep: 50, + svgMarginX: 16, + svgMarginY: 16, + stageSep: 40, graphPrefix: "graph_", nodePrefix: "node_", stagePrefix: "stage_", @@ -63,14 +63,16 @@ var VizConstants = { }; var JobPageVizConstants = { - clusterLabelSize: 11, - stageClusterLabelSize: 14 -} + clusterLabelSize: 12, + stageClusterLabelSize: 14, + rankSep: 40 +}; var StagePageVizConstants = { clusterLabelSize: 14, - stageClusterLabelSize: 18 -} + stageClusterLabelSize: 14, + rankSep: 40 +}; /* * Show or hide the RDD DAG visualization. @@ -149,11 +151,11 @@ function renderDagVizForStage(svgContainer) { var dot = metadata.select(".dot-file").text(); var containerId = VizConstants.graphPrefix + metadata.attr("stage-id"); var container = svgContainer.append("g").attr("id", containerId); - renderDot(dot, container); + renderDot(dot, container, StagePageVizConstants.rankSep); - // Round corners on RDDs + // Round corners on rectangles svgContainer - .selectAll("g.node rect") + .selectAll("rect") .attr("rx", "5") .attr("ry", "5"); } @@ -207,7 +209,13 @@ function renderDagVizForJob(svgContainer) { } // Actually render the stage - renderDot(dot, container); + renderDot(dot, container, JobPageVizConstants.rankSep); + + // Round corners on rectangles + container + .selectAll("rect") + .attr("rx", "4") + .attr("ry", "4"); // If there are any incoming edges into this graph, keep track of them to render // them separately later. Note that we cannot draw them now because we need to @@ -223,12 +231,13 @@ function renderDagVizForJob(svgContainer) { } /* Render the dot file as an SVG in the given container. */ -function renderDot(dot, container) { +function renderDot(dot, container, rankSep) { var escaped_dot = dot .replace(/</g, "<") .replace(/>/g, ">") .replace(/"/g, "\""); var g = graphlibDot.read(escaped_dot); + g.graph().rankSep = rankSep; var renderer = new dagreD3.render(); renderer(container, g); } @@ -248,12 +257,13 @@ function metadataContainer() { return d3.select("#dag-viz-metadata"); } * In general, the clustering support for dagre-d3 is quite limited at this point. */ function drawClusterLabels(svgContainer, forJob) { + var clusterLabelSize, stageClusterLabelSize; if (forJob) { - var clusterLabelSize = JobPageVizConstants.clusterLabelSize; - var stageClusterLabelSize = JobPageVizConstants.stageClusterLabelSize; + clusterLabelSize = JobPageVizConstants.clusterLabelSize; + stageClusterLabelSize = JobPageVizConstants.stageClusterLabelSize; } else { - var clusterLabelSize = StagePageVizConstants.clusterLabelSize; - var stageClusterLabelSize = StagePageVizConstants.stageClusterLabelSize; + clusterLabelSize = StagePageVizConstants.clusterLabelSize; + stageClusterLabelSize = StagePageVizConstants.stageClusterLabelSize; } svgContainer.selectAll("g.cluster").each(function() { var cluster = d3.select(this); @@ -283,7 +293,7 @@ function drawClusterLabel(d3cluster, fontSize) { .attr("x", labelX) .attr("y", labelY) .attr("text-anchor", "end") - .style("font-size", fontSize) + .style("font-size", fontSize + "px") .text(labelText); } @@ -303,12 +313,12 @@ function resizeSvg(svg) { })); var endX = VizConstants.svgMarginX + toFloat(d3.max(allClusters, function(e) { - var t = d3.select(e) + var t = d3.select(e); return getAbsolutePosition(t).x + toFloat(t.attr("width")); })); var endY = VizConstants.svgMarginY + toFloat(d3.max(allClusters, function(e) { - var t = d3.select(e) + var t = d3.select(e); return getAbsolutePosition(t).y + toFloat(t.attr("height")); })); var width = endX - startX; @@ -338,7 +348,7 @@ function drawCrossStageEdges(edges, svgContainer) { if (!dagreD3Marker.empty()) { svgContainer .append(function() { return dagreD3Marker.node().cloneNode(true); }) - .attr("id", "marker-arrow") + .attr("id", "marker-arrow"); svgContainer.selectAll("g > path").attr("marker-end", "url(#marker-arrow)"); svgContainer.selectAll("g.edgePaths def").remove(); // We no longer need these } @@ -394,12 +404,13 @@ function connectRDDs(fromRDDId, toRDDId, edgesContainer, svgContainer) { toPos.x += delta; } + var points; if (fromPos.y == toPos.y) { // If they are on the same rank, curve the middle part of the edge // upward a little to avoid interference with things in between // e.g. _______ // _____/ \_____ - var points = [ + points = [ [fromPos.x, fromPos.y], [fromPos.x + (toPos.x - fromPos.x) * 0.2, fromPos.y], [fromPos.x + (toPos.x - fromPos.x) * 0.3, fromPos.y - 20], @@ -413,7 +424,7 @@ function connectRDDs(fromRDDId, toRDDId, edgesContainer, svgContainer) { // / // | // _____/ - var points = [ + points = [ [fromPos.x, fromPos.y], [fromPos.x + (toPos.x - fromPos.x) * 0.4, fromPos.y], [fromPos.x + (toPos.x - fromPos.x) * 0.6, toPos.y], diff --git a/core/src/main/resources/org/apache/spark/ui/static/timeline-view.css b/core/src/main/resources/org/apache/spark/ui/static/timeline-view.css index 35ef14e5aaf1a..d40de704229c3 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/timeline-view.css +++ b/core/src/main/resources/org/apache/spark/ui/static/timeline-view.css @@ -23,6 +23,10 @@ div#application-timeline, div#job-timeline { margin-top: 5px; } +.vis.timeline { + line-height: 14px; +} + .vis.timeline div.content { width: 100%; } @@ -32,48 +36,55 @@ div#application-timeline, div#job-timeline { } .vis.timeline .item.stage.succeeded { - background-color: #D5DDF6; + background-color: #A0DFFF; + border-color: #3EC0FF; } .vis.timeline .item.stage.succeeded.selected { - background-color: #D5DDF6; - border-color: #97B0F8; - z-index: auto; + background-color: #A0DFFF; + border-color: #3EC0FF; + z-index: auto; } .legend-area rect.completed-stage-legend { - fill: #D5DDF6; - stroke: #97B0F8; + fill: #A0DFFF; + stroke: #3EC0FF; } .vis.timeline .item.stage.failed { - background-color: #FF5475; + background-color: #FFA1B0; + border-color: #FF4D6D; } .vis.timeline .item.stage.failed.selected { - background-color: #FF5475; - border-color: #97B0F8; - z-index: auto; + background-color: #FFA1B0; + border-color: #FF4D6D; + z-index: auto; } .legend-area rect.failed-stage-legend { - fill: #FF5475; - stroke: #97B0F8; + fill: #FFA1B0; + stroke: #FF4D6D; } .vis.timeline .item.stage.running { - background-color: #FDFFCA; + background-color: #A2FCC0; + border-color: #36F572; } .vis.timeline .item.stage.running.selected { - background-color: #FDFFCA; - border-color: #97B0F8; - z-index: auto; + background-color: #A2FCC0; + border-color: #36F572; + z-index: auto; } .legend-area rect.active-stage-legend { - fill: #FDFFCA; - stroke: #97B0F8; + fill: #A2FCC0; + stroke: #36F572; +} + +.vis.timeline .foreground { + cursor: move; } .vis.timeline .item.job { @@ -81,76 +92,81 @@ div#application-timeline, div#job-timeline { } .vis.timeline .item.job.succeeded { - background-color: #D5DDF6; + background-color: #A0DFFF; + border-color: #3EC0FF; } .vis.timeline .item.job.succeeded.selected { - background-color: #D5DDF6; - border-color: #97B0F8; - z-index: auto; + background-color: #A0DFFF; + border-color: #3EC0FF; + z-index: auto; } .legend-area rect.succeeded-job-legend { - fill: #D5DDF6; - stroke: #97B0F8; + fill: #A0DFFF; + stroke: #3EC0FF; } .vis.timeline .item.job.failed { - background-color: #FF5475; + background-color: #FFA1B0; + border-color: #FF4D6D; } .vis.timeline .item.job.failed.selected { - background-color: #FF5475; - border-color: #97B0F8; - z-index: auto; + background-color: #FFA1B0; + border-color: #FF4D6D; + z-index: auto; } .legend-area rect.failed-job-legend { - fill: #FF5475; - stroke: #97B0F8; + fill: #FFA1B0; + stroke: #FF4D6D; } .vis.timeline .item.job.running { - background-color: #FDFFCA; + background-color: #A2FCC0; + border-color: #36F572; } .vis.timeline .item.job.running.selected { - background-color: #FDFFCA; - border-color: #97B0F8; - z-index: auto; + background-color: #A2FCC0; + border-color: #36F572; + z-index: auto; } .legend-area rect.running-job-legend { - fill: #FDFFCA; - stroke: #97B0F8; + fill: #A2FCC0; + stroke: #36F572; } .vis.timeline .item.executor.added { - background-color: #D5DDF6; + background-color: #A0DFFF; + border-color: #3EC0FF; } .legend-area rect.executor-added-legend { - fill: #D5DDF6; - stroke: #97B0F8; + fill: #A0DFFF; + stroke: #3EC0FF; } .vis.timeline .item.executor.removed { - background-color: #EBCA59; + background-color: #FFA1B0; + border-color: #FF4D6D; } .legend-area rect.executor-removed-legend { - fill: #EBCA59; - stroke: #97B0F8; + fill: #FFA1B0; + stroke: #FF4D6D; } .vis.timeline .item.executor.selected { - border-color: #FFC200; - background-color: #FFF785; + background-color: #A2FCC0; + border-color: #36F572; z-index: 2; } -tr.corresponding-item-hover>td, tr.corresponding-item-hover>th { - background-color: #FFE1FA !important; +tr.corresponding-item-hover > td, tr.corresponding-item-hover > th { + background-color: #D6FFE4 !important; } #application-timeline.collapsed { @@ -165,11 +181,15 @@ tr.corresponding-item-hover>td, tr.corresponding-item-hover>th { margin-bottom: 5px; } +.control-panel input[type="checkbox"] { + margin: 0; +} + span.expand-application-timeline, span.expand-job-timeline { cursor: pointer; } -.control-panel input+span { +.control-panel input + span { cursor: pointer; } @@ -180,3 +200,17 @@ span.expand-application-timeline, span.expand-job-timeline { .vis.timeline .item .tooltip-inner { max-width: unset !important; } + +.vispanel.center { + font-size: 12px; + line-height: 12px; +} + +.legend-area text { + fill: #4D4D4D; +} + +.additional-metrics ul { + list-style: none; + margin-left: 15px; +} diff --git a/core/src/main/resources/org/apache/spark/ui/static/timeline-view.js b/core/src/main/resources/org/apache/spark/ui/static/timeline-view.js index e4a891d47f035..48fbb33b1155b 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/timeline-view.js +++ b/core/src/main/resources/org/apache/spark/ui/static/timeline-view.js @@ -156,9 +156,9 @@ function setupExecutorEventAction() { function setupZoomable(id, timeline) { $(id + '>input[type="checkbox"]').click(function() { if (this.checked) { - timeline.setOptions({zoomable: false}); - } else { timeline.setOptions({zoomable: true}); + } else { + timeline.setOptions({zoomable: false}); } }); diff --git a/core/src/main/resources/org/apache/spark/ui/static/webui.css b/core/src/main/resources/org/apache/spark/ui/static/webui.css index 669ad48937c05..e7c1d475d4e52 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/webui.css +++ b/core/src/main/resources/org/apache/spark/ui/static/webui.css @@ -106,14 +106,18 @@ span.rest-uri { } pre { - font-size: 0.8em; + font-size: 12px; + line-height: 18px; + padding: 6px; + margin: 0; + border-radius: 3px; } .stage-details { max-height: 100px; overflow-y: auto; margin: 0; - transition: max-height 0.5s ease-out, padding 0.5s ease-out; + transition: max-height 0.25s ease-out, padding 0.25s ease-out; } .stage-details.collapsed { @@ -135,7 +139,7 @@ pre { max-height: 300px; overflow-y: auto; margin: 0; - transition: max-height 0.5s ease-out, padding 0.5s ease-out; + transition: max-height 0.25s ease-out, padding 0.25s ease-out; } .stacktrace-details.collapsed { @@ -158,7 +162,7 @@ span.additional-metric-title { } .tooltip { - font-weight: normal; + font-weight: normal; } .arrow-open { @@ -166,9 +170,9 @@ span.additional-metric-title { height: 0; border-left: 5px solid transparent; border-right: 5px solid transparent; - border-top: 5px solid black; - float: left; - margin-top: 6px; + border-top: 5px solid #08c; + display: inline-block; + margin-bottom: 2px; } .arrow-closed { @@ -176,8 +180,10 @@ span.additional-metric-title { height: 0; border-top: 5px solid transparent; border-bottom: 5px solid transparent; - border-left: 5px solid black; + border-left: 5px solid #08c; display: inline-block; + margin-left: 2px; + margin-right: 3px; } .version { @@ -196,3 +202,17 @@ span.additional-metric-title { .serialization_time, .getting_result_time { display: none; } + +.accordion-inner { + background: #f5f5f5; +} + +.accordion-inner pre { + border: 0; + padding: 0; + background: none; +} + +a.expandbutton { + cursor: pointer; +} diff --git a/core/src/main/scala/org/apache/spark/ui/ToolTips.scala b/core/src/main/scala/org/apache/spark/ui/ToolTips.scala index 24f3236456248..063e2a1f8b18e 100644 --- a/core/src/main/scala/org/apache/spark/ui/ToolTips.scala +++ b/core/src/main/scala/org/apache/spark/ui/ToolTips.scala @@ -57,4 +57,23 @@ private[spark] object ToolTips { val GC_TIME = """Time that the executor spent paused for Java garbage collection while the task was running.""" + + val JOB_TIMELINE = + """Shows when jobs started and ended and when executors joined or left. Drag to scroll. + Click Enable Zooming and use mouse wheel to zoom in/out.""" + + val STAGE_TIMELINE = + """Shows when stages started and ended and when executors joined or left. Drag to scroll. + Click Enable Zooming and use mouse wheel to zoom in/out.""" + + val JOB_DAG = + """Shows a graph of stages executed for this job, each of which can contain + multiple RDD operations (e.g. map() and filter()), and of RDDs inside each operation + (shown as dots).""" + + val STAGE_DAG = + """Shows a graph of RDD operations in this stage, and RDDs inside each one. A stage can run + multiple operations (e.g. two map() functions) if they can be pipelined. Some operations + also create multiple RDDs internally. Cached RDDs are shown in green. + """ } diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala index 97eed13c2d780..6a0f5c5d16daa 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -156,10 +156,10 @@ private[spark] object UIUtils extends Logging { def commonHeaderNodes: Seq[Node] = { - - - - + + + + @@ -250,7 +250,7 @@ private[spark] object UIUtils extends Logging {

      - {org.apache.spark.SPARK_VERSION} {title} @@ -350,7 +350,10 @@ private[spark] object UIUtils extends Logging {
      - DAG visualization + + DAG Visualization +
      diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala index 09323d1d80ad6..e010ebef3b34a 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala @@ -18,12 +18,12 @@ package org.apache.spark.ui.jobs import scala.collection.mutable.{HashMap, ListBuffer} -import scala.xml.{Node, NodeSeq, Unparsed} +import scala.xml.{Node, NodeSeq, Unparsed, Utility} import java.util.Date import javax.servlet.http.HttpServletRequest -import org.apache.spark.ui.{UIUtils, WebUIPage} +import org.apache.spark.ui.{ToolTips, UIUtils, WebUIPage} import org.apache.spark.ui.jobs.UIData.{ExecutorUIData, JobUIData} import org.apache.spark.JobExecutionStatus @@ -81,6 +81,9 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { case JobExecutionStatus.RUNNING => "running" } + // The timeline library treats contents as HTML, so we have to escape them; for the + // data-title attribute string we have to escape them twice since that's in a string. + val escapedDesc = Utility.escape(displayJobDescription) val jobEventJsonAsStr = s""" |{ @@ -90,16 +93,17 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { | 'end': new Date(${completionTime}), | 'content': '
      ' + + | 'Status: ${status}
      ' + + | 'Submitted: ${UIUtils.formatDate(new Date(submissionTime))}' + | '${ if (status != JobExecutionStatus.RUNNING) { - s"""
      Completion Time: ${UIUtils.formatDate(new Date(completionTime))}""" + s"""
      Completed: ${UIUtils.formatDate(new Date(completionTime))}""" } else { "" } }">' + - | '${displayJobDescription} (Job ${jobId})
      ' + | '${escapedDesc} (Job ${jobId})
      ' |} """.stripMargin jobEventJsonAsStr @@ -179,13 +183,15 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { - Event timeline + + Event Timeline + ++ ++ @@ -283,7 +289,7 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { {if (parent.sc.isDefined) { // Total duration is not meaningful unless the UI is live
    • - Total Duration: + Total Uptime: {UIUtils.formatDuration(System.currentTimeMillis() - startTime)}
    • }} @@ -336,9 +342,8 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { failedJobsTable } - val helpText = """A job is triggered by an action, like "count()" or "saveAsTextFile()".""" + - " Click on a job's title to see information about the stages of tasks associated with" + - " the job." + val helpText = """A job is triggered by an action, like count() or saveAsTextFile().""" + + " Click on a job to see information about the stages of tasks inside it." UIUtils.headerSparkPage("Spark Jobs", content, parent, helpText = Some(helpText)) } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala index a37f739ab9c66..5e52942b64f3f 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala @@ -73,19 +73,6 @@ private[ui] class AllStagesPage(parent: StagesTab) extends WebUIPage("") { val summary: NodeSeq =
        - { - if (sc.isDefined) { - // Total duration is not meaningful unless the UI is live -
      • - Total Duration: - {UIUtils.formatDuration(now - sc.get.startTime)} -
      • - } - } -
      • - Scheduling Mode: - {listener.schedulingMode.map(_.toString).getOrElse("Unknown")} -
      • { if (shouldShowActiveStages) {
      • @@ -145,7 +132,7 @@ private[ui] class AllStagesPage(parent: StagesTab) extends WebUIPage("") { content ++=

        Failed Stages ({numFailedStages})

        ++ failedStagesTable.toNodeSeq } - UIUtils.headerSparkPage("Spark Stages (for all jobs)", content, parent) + UIUtils.headerSparkPage("Stages for All Jobs", content, parent) } } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala index 7163217e1fed0..2cad0a796913e 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala @@ -20,13 +20,13 @@ package org.apache.spark.ui.jobs import java.util.Date import scala.collection.mutable.{Buffer, HashMap, ListBuffer} -import scala.xml.{NodeSeq, Node, Unparsed} +import scala.xml.{NodeSeq, Node, Unparsed, Utility} import javax.servlet.http.HttpServletRequest import org.apache.spark.JobExecutionStatus import org.apache.spark.scheduler.StageInfo -import org.apache.spark.ui.{UIUtils, WebUIPage} +import org.apache.spark.ui.{ToolTips, UIUtils, WebUIPage} import org.apache.spark.ui.jobs.UIData.ExecutorUIData /** Page showing statistics and stage list for a given job */ @@ -64,6 +64,9 @@ private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") { val submissionTime = stage.submissionTime.get val completionTime = stage.completionTime.getOrElse(System.currentTimeMillis()) + // The timeline library treats contents as HTML, so we have to escape them; for the + // data-title attribute string we have to escape them twice since that's in a string. + val escapedName = Utility.escape(name) s""" |{ | 'className': 'stage job-timeline-object ${status}', @@ -72,17 +75,17 @@ private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") { | 'end': new Date(${completionTime}), | 'content': '
        ' + | 'Status: ${status.toUpperCase}
        ' + - | 'Submission Time: ${UIUtils.formatDate(new Date(submissionTime))}' + + | 'Submitted: ${UIUtils.formatDate(new Date(submissionTime))}' + | '${ if (status != "running") { - s"""
        Completion Time: ${UIUtils.formatDate(new Date(completionTime))}""" + s"""
        Completed: ${UIUtils.formatDate(new Date(completionTime))}""" } else { "" } }">' + - | '${name} (Stage ${stageId}.${attemptId})
        ', + | '${escapedName} (Stage ${stageId}.${attemptId})
      ', |} """.stripMargin } @@ -161,13 +164,15 @@ private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") { - Event timeline + + Event Timeline + ++ ++ diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index b01fad8e453c8..8f7b1c2f09665 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -81,7 +81,7 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") {
      • - Total task time across all tasks: + Total Time Across All Tasks: {UIUtils.formatDuration(stageData.executorRunTime)}
      • {if (stageData.hasInput) { @@ -98,25 +98,25 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { }} {if (stageData.hasShuffleRead) {
      • - Shuffle read: + Shuffle Read: {s"${Utils.bytesToString(stageData.shuffleReadTotalBytes)} / " + s"${stageData.shuffleReadRecords}"}
      • }} {if (stageData.hasShuffleWrite) {
      • - Shuffle write: + Shuffle Write: {s"${Utils.bytesToString(stageData.shuffleWriteBytes)} / " + s"${stageData.shuffleWriteRecords}"}
      • }} {if (stageData.hasBytesSpilled) {
      • - Shuffle spill (memory): + Shuffle Spill (Memory): {Utils.bytesToString(stageData.memoryBytesSpilled)}
      • - Shuffle spill (disk): + Shuffle Spill (Disk): {Utils.bytesToString(stageData.diskBytesSpilled)}
      • }} @@ -127,10 +127,10 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") {
        - Show additional metrics + Show Additional Metrics